[HUDI-2177][HUDI-2200] Adding virtual keys support for MOR table (#3315)
This commit is contained in:
committed by
GitHub
parent
dde57b293c
commit
fe508376fa
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
@@ -67,7 +69,7 @@ public class TestHoodieRealtimeFileSplit {
|
||||
baseFileSplit = new FileSplit(new Path(fileSplitName), 0, 100, new String[] {});
|
||||
maxCommitTime = "10001";
|
||||
|
||||
split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogPaths, maxCommitTime);
|
||||
split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogPaths, maxCommitTime, Option.empty());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -97,6 +99,7 @@ public class TestHoodieRealtimeFileSplit {
|
||||
inorder.verify(out, times(1)).writeInt(eq(deltaLogPaths.size()));
|
||||
inorder.verify(out, times(1)).writeInt(eq(deltaLogPaths.get(0).length()));
|
||||
inorder.verify(out, times(1)).write(aryEq(deltaLogPaths.get(0).getBytes(StandardCharsets.UTF_8)));
|
||||
inorder.verify(out, times(1)).writeBoolean(false);
|
||||
// verify there are no more interactions happened on the mocked object
|
||||
inorder.verifyNoMoreInteractions();
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
@@ -31,6 +32,7 @@ import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.SchemaTestUtil;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
|
||||
@@ -69,6 +71,7 @@ import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
@@ -119,6 +122,16 @@ public class TestHoodieRealtimeRecordReader {
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveOrderedColumnNames);
|
||||
}
|
||||
|
||||
protected Properties getPropertiesForKeyGen() {
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
|
||||
properties.put("hoodie.datasource.write.recordkey.field","_row_key");
|
||||
properties.put("hoodie.datasource.write.partitionpath.field","partition_path");
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key(), "_row_key");
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key(), "partition_path");
|
||||
return properties;
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("testArguments")
|
||||
public void testReader(ExternalSpillableMap.DiskMapType diskMapType,
|
||||
@@ -175,7 +188,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(h -> h.getPath().toString()).collect(Collectors.toList()),
|
||||
instantTime);
|
||||
instantTime, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -246,7 +259,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
String logFilePath = writer.getLogFile().getPath().toString();
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime);
|
||||
basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -326,7 +339,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
String logFilePath = writer.getLogFile().getPath().toString();
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime);
|
||||
basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -472,7 +485,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
// create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), logFilePaths, newCommitTime);
|
||||
basePath.toUri().toString(), logFilePaths, newCommitTime, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
|
||||
Reference in New Issue
Block a user