-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1722]hive beeline/spark-sql query specified field on mor table occur NPE #2722
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -156,6 +156,73 @@ public void multiLevelPartitionReadersRealtimeCombineHoodieInputFormat() throws | |
| assertEquals(3000, counter); | ||
| } | ||
|
|
||
| @Test | ||
| public void testMutilReaderRealtimeComineHoodieInputFormat() throws Exception { | ||
| // test for hudi-1722 | ||
| Configuration conf = new Configuration(); | ||
| // initial commit | ||
| Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); | ||
| HoodieTestUtils.init(hadoopConf, tempDir.toAbsolutePath().toString(), HoodieTableType.MERGE_ON_READ); | ||
| String commitTime = "100"; | ||
| final int numRecords = 1000; | ||
| // Create 3 parquet files with 1000 records each | ||
| File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); | ||
| InputFormatTestUtil.commit(tempDir, commitTime); | ||
|
|
||
| String newCommitTime = "101"; | ||
| // to trigger the bug of HUDI-1772, only update fileid2 | ||
| // insert 1000 update records to log file 2 | ||
| // now fileid0, fileid1 has no log files, fileid2 has log file | ||
| HoodieLogFormat.Writer writer = | ||
| InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid2", commitTime, newCommitTime, | ||
| numRecords, numRecords, 0); | ||
| writer.close(); | ||
|
|
||
| TableDesc tblDesc = Utilities.defaultTd; | ||
| // Set the input format | ||
| tblDesc.setInputFileFormatClass(HoodieParquetRealtimeInputFormat.class); | ||
| PartitionDesc partDesc = new PartitionDesc(tblDesc, null); | ||
| LinkedHashMap<Path, PartitionDesc> pt = new LinkedHashMap<>(); | ||
| LinkedHashMap<Path, ArrayList<String>> tableAlias = new LinkedHashMap<>(); | ||
| ArrayList<String> alias = new ArrayList<>(); | ||
| alias.add(tempDir.toAbsolutePath().toString()); | ||
| tableAlias.put(new Path(tempDir.toAbsolutePath().toString()), alias); | ||
| pt.put(new Path(tempDir.toAbsolutePath().toString()), partDesc); | ||
|
|
||
| MapredWork mrwork = new MapredWork(); | ||
| mrwork.getMapWork().setPathToPartitionInfo(pt); | ||
| mrwork.getMapWork().setPathToAliases(tableAlias); | ||
| Path mapWorkPath = new Path(tempDir.toAbsolutePath().toString()); | ||
| Utilities.setMapRedWork(conf, mrwork, mapWorkPath); | ||
| jobConf = new JobConf(conf); | ||
| // Add the paths | ||
| FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); | ||
| jobConf.set(HAS_MAP_WORK, "true"); | ||
| // The following config tells Hive to choose ExecMapper to read the MAP_WORK | ||
| jobConf.set(MAPRED_MAPPER_CLASS, ExecMapper.class.getName()); | ||
| // set SPLIT_MAXSIZE larger to create one split for 3 files groups | ||
| jobConf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.SPLIT_MAXSIZE, "128000000"); | ||
|
|
||
| HoodieCombineHiveInputFormat combineHiveInputFormat = new HoodieCombineHiveInputFormat(); | ||
| String tripsHiveColumnTypes = "double,string,string,string,double,double,double,double,double"; | ||
| InputFormatTestUtil.setProjectFieldsForInputFormat(jobConf, schema, tripsHiveColumnTypes); | ||
| InputSplit[] splits = combineHiveInputFormat.getSplits(jobConf, 1); | ||
| // Since the SPLIT_SIZE is 3, we should create only 1 split with all 3 file groups | ||
| assertEquals(1, splits.length); | ||
| RecordReader<NullWritable, ArrayWritable> recordReader = | ||
|
||
| combineHiveInputFormat.getRecordReader(splits[0], jobConf, null); | ||
| NullWritable nullWritable = recordReader.createKey(); | ||
| ArrayWritable arrayWritable = recordReader.createValue(); | ||
| int counter = 0; | ||
| while (recordReader.next(nullWritable, arrayWritable)) { | ||
| // read over all the splits | ||
| counter++; | ||
| } | ||
| // should read out 3 splits, each for file0, file1, file2 containing 1000 records each | ||
| assertEquals(3000, counter); | ||
| recordReader.close(); | ||
| } | ||
|
|
||
| @Test | ||
| @Disabled | ||
| public void testHoodieRealtimeCombineHoodieInputFormat() throws Exception { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.