- Ugrading to Hive 2.x
- Eliminating in-memory deltaRecordsMap - Use writerSchema to generate generic record needed by custom payloads - changes to make tests work with hive 2.x
This commit is contained in:
committed by
vinoth chandar
parent
cd7623e216
commit
129e433641
@@ -26,6 +26,7 @@ import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
@@ -214,9 +215,9 @@ public class HoodieInputFormatTest {
|
||||
int totalCount = 0;
|
||||
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
|
||||
for (InputSplit split : splits) {
|
||||
RecordReader<Void, ArrayWritable> recordReader = inputFormat
|
||||
RecordReader<NullWritable, ArrayWritable> recordReader = inputFormat
|
||||
.getRecordReader(split, jobConf, null);
|
||||
Void key = recordReader.createKey();
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable writable = recordReader.createValue();
|
||||
|
||||
while (recordReader.next(key, writable)) {
|
||||
|
||||
@@ -20,7 +20,9 @@ package com.uber.hoodie.hadoop;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
import java.io.File;
|
||||
import java.io.FilenameFilter;
|
||||
@@ -29,8 +31,10 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.parquet.avro.AvroParquetWriter;
|
||||
@@ -79,6 +83,11 @@ public class InputFormatTestUtil {
|
||||
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".commit").createNewFile();
|
||||
}
|
||||
|
||||
public static void deltaCommit(TemporaryFolder basePath, String commitNumber) throws IOException {
|
||||
// create the commit
|
||||
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".deltacommit").createNewFile();
|
||||
}
|
||||
|
||||
public static void setupIncremental(JobConf jobConf, String startCommit,
|
||||
int numberOfCommitsToPull) {
|
||||
String modePropertyName = String
|
||||
@@ -107,6 +116,16 @@ public class InputFormatTestUtil {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
|
||||
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema,
|
||||
int numberOfFiles, int numberOfRecords, String commitNumber) throws Exception {
|
||||
basePath.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema,
|
||||
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
|
||||
baseDir.create();
|
||||
@@ -135,6 +154,31 @@ public class InputFormatTestUtil {
|
||||
}
|
||||
}
|
||||
|
||||
private static void createSimpleData(Schema schema,
|
||||
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
|
||||
throws Exception {
|
||||
AvroParquetWriter parquetWriter;
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i);
|
||||
File dataFile = new File(partitionPath, fileId);
|
||||
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
|
||||
try {
|
||||
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, numberOfRecords);
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
for (IndexedRecord record : records) {
|
||||
GenericRecord p = HoodieAvroUtils.rewriteRecord((GenericRecord) record, hoodieFieldsSchema);
|
||||
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
|
||||
p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
|
||||
p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitNumber);
|
||||
parquetWriter.write(p);
|
||||
}
|
||||
} finally {
|
||||
parquetWriter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static Iterable<? extends GenericRecord> generateAvroRecords(Schema schema,
|
||||
int numberOfRecords, String commitTime, String fileId) throws IOException {
|
||||
List<GenericRecord> records = new ArrayList<>(numberOfRecords);
|
||||
|
||||
@@ -48,6 +48,7 @@ import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -60,6 +61,7 @@ import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
@@ -91,7 +93,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, int numberOfRecords)
|
||||
throws InterruptedException, IOException {
|
||||
return writeLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0, 0);
|
||||
return writeDataBlockToLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0, 0);
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer writeRollback(File partitionDir, Schema schema, String fileId,
|
||||
@@ -115,7 +117,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
return writer;
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId,
|
||||
private HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, int numberOfRecords, int offset, int logVersion)
|
||||
throws InterruptedException, IOException {
|
||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
@@ -137,6 +139,25 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
return writer;
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, String oldCommit, int logVersion)
|
||||
throws InterruptedException, IOException {
|
||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(partitionDir.getPath()))
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
|
||||
.overBaseCommit(baseCommit).withLogVersion(logVersion).withFs(fs).build();
|
||||
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
|
||||
.ordinal()));
|
||||
HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header);
|
||||
writer = writer.appendBlock(rollbackBlock);
|
||||
return writer;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReader() throws Exception {
|
||||
testReader(true);
|
||||
@@ -155,7 +176,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
String baseInstant = "100";
|
||||
File partitionDir =
|
||||
partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
|
||||
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
|
||||
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
|
||||
InputFormatTestUtil.commit(basePath, baseInstant);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
@@ -183,7 +204,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant,
|
||||
instantTime, String.valueOf(baseInstantTs + logVersion - 1), logVersion);
|
||||
} else {
|
||||
writer = writeLogFile(partitionDir, schema, "fileid0", baseInstant,
|
||||
writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant,
|
||||
instantTime, 100, 0, logVersion);
|
||||
}
|
||||
long size = writer.getCurrentSize();
|
||||
@@ -199,7 +220,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
.collect(Collectors.toList()), instantTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<Void, ArrayWritable> reader =
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
@@ -219,7 +240,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
//use reader to read base Parquet File and log file, merge in flight and return latest commit
|
||||
//here all 100 records should be updated, see above
|
||||
Void key = recordReader.createKey();
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
while (recordReader.next(key, value)) {
|
||||
Writable[] values = value.get();
|
||||
@@ -255,7 +276,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
// insert new records to log file
|
||||
String newCommitTime = "101";
|
||||
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, numRecords, numRecords, 0);
|
||||
long size = writer.getCurrentSize();
|
||||
writer.close();
|
||||
@@ -268,7 +289,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<Void, ArrayWritable> reader =
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
@@ -288,7 +309,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
//use reader to read base Parquet File and log file
|
||||
//here all records should be present. Also ensure log records are in order.
|
||||
Void key = recordReader.createKey();
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
int numRecordsAtCommit1 = 0;
|
||||
int numRecordsAtCommit2 = 0;
|
||||
@@ -343,6 +364,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
long size = writer.getCurrentSize();
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
|
||||
|
||||
//create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
String logFilePath = writer.getLogFile().getPath().toString();
|
||||
@@ -351,7 +373,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<Void, ArrayWritable> reader =
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
@@ -370,7 +392,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
// use reader to read base Parquet File and log file, merge in flight and return latest commit
|
||||
// here the first 50 records should be updated, see above
|
||||
Void key = recordReader.createKey();
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
int numRecordsRead = 0;
|
||||
while (recordReader.next(key, value)) {
|
||||
@@ -420,26 +442,26 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
// Assert type MAP
|
||||
ArrayWritable mapItem = (ArrayWritable) values[12];
|
||||
Writable[] mapItemValues = ((ArrayWritable) mapItem.get()[0]).get();
|
||||
ArrayWritable mapItemValue1 = (ArrayWritable) mapItemValues[0];
|
||||
ArrayWritable mapItemValue2 = (ArrayWritable) mapItemValues[1];
|
||||
Assert.assertEquals("test value for field: tags", mapItemValue1.get()[0].toString(),
|
||||
Writable mapItemValue1 = mapItem.get()[0];
|
||||
Writable mapItemValue2 = mapItem.get()[1];
|
||||
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get()[0].toString(),
|
||||
"mapItem1");
|
||||
Assert.assertEquals("test value for field: tags", mapItemValue2.get()[0].toString(),
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get()[0].toString(),
|
||||
"mapItem2");
|
||||
ArrayWritable mapItemValue1value = (ArrayWritable) mapItemValue1.get()[1];
|
||||
ArrayWritable mapItemValue2value = (ArrayWritable) mapItemValue2.get()[1];
|
||||
Assert.assertEquals("test value for field: tags", mapItemValue1value.get().length, 2);
|
||||
Assert.assertEquals("test value for field: tags", mapItemValue2value.get().length, 2);
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get().length, 2);
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get().length, 2);
|
||||
Writable mapItemValue1value = ((ArrayWritable) mapItemValue1).get()[1];
|
||||
Writable mapItemValue2value = ((ArrayWritable) mapItemValue2).get()[1];
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item1",
|
||||
mapItemValue1value.get()[0].toString(), "item" + currentRecordNo);
|
||||
((ArrayWritable) mapItemValue1value).get()[0].toString(), "item" + currentRecordNo);
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1",
|
||||
mapItemValue2value.get()[0].toString(), "item2" + currentRecordNo);
|
||||
((ArrayWritable) mapItemValue2value).get()[0].toString(), "item2" + currentRecordNo);
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2",
|
||||
mapItemValue1value.get()[1].toString(),
|
||||
((ArrayWritable) mapItemValue1value).get()[1].toString(),
|
||||
"item" + currentRecordNo + recordCommitTimeSuffix);
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2",
|
||||
mapItemValue2value.get()[1].toString(),
|
||||
((ArrayWritable) mapItemValue2value).get()[1].toString(),
|
||||
"item2" + currentRecordNo + recordCommitTimeSuffix);
|
||||
|
||||
// Assert type RECORD
|
||||
@@ -453,11 +475,96 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
// Assert type ARRAY
|
||||
ArrayWritable arrayValue = (ArrayWritable) values[14];
|
||||
Writable[] arrayValues = ((ArrayWritable) arrayValue.get()[0]).get();
|
||||
Writable[] arrayValues = arrayValue.get();
|
||||
for (int i = 0; i < arrayValues.length; i++) {
|
||||
Assert.assertEquals("test value for field: stringArray", "stringArray" + i + recordCommitTimeSuffix,
|
||||
arrayValues[i].toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaEvolutionAndRollbackBlockInLastLogFile() throws Exception {
|
||||
// initial commit
|
||||
List<String> logFilePaths = new ArrayList<>();
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
int numberOfRecords = 100;
|
||||
int numberOfLogRecords = numberOfRecords / 2;
|
||||
File partitionDir = InputFormatTestUtil
|
||||
.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
|
||||
InputFormatTestUtil.commit(basePath, commitTime);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
List<Field> firstSchemaFields = schema.getFields();
|
||||
|
||||
// update files and generate new log file but don't commit
|
||||
schema = SchemaTestUtil.getComplexEvolvedSchema();
|
||||
String newCommitTime = "101";
|
||||
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, numberOfLogRecords, 0, 1);
|
||||
long size = writer.getCurrentSize();
|
||||
logFilePaths.add(writer.getLogFile().getPath().toString());
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
|
||||
// write rollback for the previous block in new log file version
|
||||
newCommitTime = "102";
|
||||
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, "101", 1);
|
||||
logFilePaths.add(writer.getLogFile().getPath().toString());
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
|
||||
|
||||
//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_" + commitTime + ".parquet"), 0, 1,
|
||||
jobConf), basePath.getRoot().getPath(), logFilePaths, newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
|
||||
assert (firstSchemaFields.containsAll(fields) == false);
|
||||
|
||||
// Try to read all the fields passed by the new schema
|
||||
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
|
||||
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
|
||||
HoodieRealtimeRecordReader recordReader = null;
|
||||
try {
|
||||
// validate record reader compaction
|
||||
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
throw new RuntimeException("should've failed the previous line");
|
||||
} catch (HoodieException e) {
|
||||
// expected, field not found since the data written with the evolved schema was rolled back
|
||||
}
|
||||
|
||||
// Try to read all the fields passed by the new schema
|
||||
names = firstSchemaFields.stream().map(f -> f.name()).collect(Collectors.joining(","));
|
||||
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
// This time read only the fields which are part of parquet
|
||||
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
// use reader to read base Parquet File and log file
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
while (recordReader.next(key, value)) {
|
||||
// keep reading
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user