Support union mode in HoodieRealtimeRecordReader for pure insert workloads
Also Replace BufferedIteratorPayload abstraction with function passing
This commit is contained in:
committed by
vinoth chandar
parent
93f345a032
commit
dfc0c61eb7
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hadoop;
|
||||
|
||||
import groovy.lang.Tuple2;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRecordReaderValueIterator {
|
||||
|
||||
@Test
|
||||
public void testValueIterator() {
|
||||
String[] values = new String[]{
|
||||
"hoodie",
|
||||
"efficient",
|
||||
"new project",
|
||||
"realtime",
|
||||
"spark",
|
||||
"dataset",
|
||||
};
|
||||
List<Tuple2<Integer, String>> entries = IntStream.range(0, values.length)
|
||||
.boxed().map(idx -> new Tuple2<>(idx, values[idx])).collect(Collectors.toList());
|
||||
TestRecordReader reader = new TestRecordReader(entries);
|
||||
RecordReaderValueIterator<IntWritable, Text> itr = new RecordReaderValueIterator<IntWritable, Text>(reader);
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Assert.assertTrue(itr.hasNext());
|
||||
Text val = itr.next();
|
||||
Assert.assertEquals(values[i], val.toString());
|
||||
}
|
||||
Assert.assertFalse(itr.hasNext());
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple replay record reader for unit-testing
|
||||
*/
|
||||
private static class TestRecordReader implements RecordReader<IntWritable, Text> {
|
||||
|
||||
private final List<Tuple2<Integer, String>> entries;
|
||||
private int currIndex = 0;
|
||||
|
||||
public TestRecordReader(List<Tuple2<Integer, String>> entries) {
|
||||
this.entries = entries;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean next(IntWritable key, Text value) throws IOException {
|
||||
if (currIndex >= entries.size()) {
|
||||
return false;
|
||||
}
|
||||
key.set(entries.get(currIndex).getFirst());
|
||||
value.set(entries.get(currIndex).getSecond());
|
||||
currIndex++;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntWritable createKey() {
|
||||
return new IntWritable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Text createValue() {
|
||||
return new Text();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
return currIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return (currIndex * 1.0F) / entries.size();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -35,8 +35,10 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -71,7 +73,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
@Before
|
||||
public void setUp() {
|
||||
jobConf = new JobConf();
|
||||
jobConf.set(HoodieRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024));
|
||||
jobConf.set(AbstractRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024));
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf);
|
||||
}
|
||||
@@ -82,12 +84,18 @@ 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);
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, int numberOfRecords, int offset)
|
||||
throws InterruptedException, IOException {
|
||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(partitionDir.getPath()))
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
|
||||
.overBaseCommit(baseCommit).withFs(fs).build();
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (int i = 0; i < numberOfRecords; i++) {
|
||||
for (int i = offset; i < offset + numberOfRecords; i++) {
|
||||
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
|
||||
}
|
||||
Schema writeSchema = records.get(0).getSchema();
|
||||
@@ -142,8 +150,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
|
||||
//validate record reader compaction
|
||||
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf,
|
||||
reader);
|
||||
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
|
||||
//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
|
||||
@@ -158,6 +165,90 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnMergedReader() throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
final int numRecords = 1000;
|
||||
final int firstBatchLastRecordKey = numRecords - 1;
|
||||
final int secondBatchLastRecordKey = 2 * numRecords - 1;
|
||||
File partitionDir = InputFormatTestUtil
|
||||
.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
|
||||
InputFormatTestUtil.commit(basePath, commitTime);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
// insert new records to log file
|
||||
String newCommitTime = "101";
|
||||
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, numRecords, numRecords);
|
||||
long size = writer.getCurrentSize();
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
|
||||
//create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
String logFilePath = writer.getLogFile().getPath().toString();
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1,
|
||||
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<Void, 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();
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
String postions = 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, postions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
// Enable merge skipping.
|
||||
jobConf.set("hoodie.realtime.merge.skip", "true");
|
||||
|
||||
//validate unmerged record reader
|
||||
RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader);
|
||||
|
||||
//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();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
int numRecordsAtCommit1 = 0;
|
||||
int numRecordsAtCommit2 = 0;
|
||||
Set<Integer> seenKeys = new HashSet<>();
|
||||
Integer lastSeenKeyFromLog = firstBatchLastRecordKey;
|
||||
while (recordReader.next(key, value)) {
|
||||
Writable[] values = value.get();
|
||||
String gotCommit = values[0].toString();
|
||||
String keyStr = values[2].toString();
|
||||
Integer gotKey = Integer.parseInt(keyStr.substring("key".length()));
|
||||
if (gotCommit.equals(newCommitTime)) {
|
||||
numRecordsAtCommit2++;
|
||||
Assert.assertTrue(gotKey > firstBatchLastRecordKey);
|
||||
Assert.assertTrue(gotKey <= secondBatchLastRecordKey);
|
||||
Assert.assertEquals(gotKey.intValue(), lastSeenKeyFromLog + 1);
|
||||
lastSeenKeyFromLog++;
|
||||
} else {
|
||||
numRecordsAtCommit1++;
|
||||
Assert.assertTrue(gotKey >= 0);
|
||||
Assert.assertTrue(gotKey <= firstBatchLastRecordKey);
|
||||
}
|
||||
// Ensure unique key
|
||||
Assert.assertFalse(seenKeys.contains(gotKey));
|
||||
seenKeys.add(gotKey);
|
||||
key = recordReader.createKey();
|
||||
value = recordReader.createValue();
|
||||
}
|
||||
Assert.assertEquals(numRecords, numRecordsAtCommit1);
|
||||
Assert.assertEquals(numRecords, numRecordsAtCommit2);
|
||||
Assert.assertEquals(2 * numRecords, seenKeys.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReaderWithNestedAndComplexSchema() throws Exception {
|
||||
// initial commit
|
||||
@@ -203,8 +294,7 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
|
||||
// validate record reader compaction
|
||||
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf,
|
||||
reader);
|
||||
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
|
||||
// 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
|
||||
|
||||
Reference in New Issue
Block a user