1
0

Support union mode in HoodieRealtimeRecordReader for pure insert workloads

Also Replace BufferedIteratorPayload abstraction with function passing
This commit is contained in:
Balaji Varadarajan
2018-04-26 10:18:05 -07:00
committed by vinoth chandar
parent 93f345a032
commit dfc0c61eb7
44 changed files with 2545 additions and 1179 deletions

View File

@@ -73,12 +73,11 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class HoodieLogFormatTest {
private static final String BASE_OUTPUT_PATH = "/tmp/";
private static String basePath;
private FileSystem fs;
private Path partitionPath;
private static String basePath;
private int bufferSize = 4096;
private static final String BASE_OUTPUT_PATH = "/tmp/";
private Boolean readBlocksLazily = true;
public HoodieLogFormatTest(Boolean readBlocksLazily) {
@@ -87,7 +86,7 @@ public class HoodieLogFormatTest {
@Parameterized.Parameters(name = "LogBlockReadMode")
public static Collection<Boolean[]> data() {
return Arrays.asList(new Boolean[][] {{true}, {false}});
return Arrays.asList(new Boolean[][]{{true}, {false}});
}
@BeforeClass
@@ -400,7 +399,7 @@ public class HoodieLogFormatTest {
writer.close();
// scan all log blocks (across multiple log files)
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath,
logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100",
10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
@@ -527,7 +526,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -587,7 +586,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -665,7 +664,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"103", 10240L, true, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -719,7 +718,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"102", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(200);
@@ -739,8 +738,8 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(commandBlock);
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily,
false, bufferSize, BASE_OUTPUT_PATH);
scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101",
10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size());
}
@@ -800,7 +799,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
// all data must be rolled back before merge
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords());
@@ -849,7 +848,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords());
}
@@ -881,7 +880,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
@@ -931,7 +930,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords());
}
@@ -1019,7 +1018,7 @@ public class HoodieLogFormatTest {
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema,
"101", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH);
assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords());
}

View File

@@ -27,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import com.uber.hoodie.common.util.SpillableMapUtils;
@@ -156,14 +157,14 @@ public class TestDiskBasedMap {
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()));
new HoodieRecordSizeEstimator(schema));
assertTrue(payloadSize > 0);
// Test sizeEstimator with hoodie metadata fields
schema = HoodieAvroUtils.addMetadataFields(schema);
hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()));
new HoodieRecordSizeEstimator(schema));
assertTrue(payloadSize > 0);
// Following tests payloads without an Avro Schema in the Record
@@ -175,7 +176,7 @@ public class TestDiskBasedMap {
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName()));
new HoodieRecordSizeEstimator(schema));
assertTrue(payloadSize > 0);
// Test sizeEstimator with hoodie metadata fields and without schema object in the payload
@@ -188,7 +189,7 @@ public class TestDiskBasedMap {
.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
.collect(Collectors.toList());
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName()));
new HoodieRecordSizeEstimator(schema));
assertTrue(payloadSize > 0);
}
@@ -201,11 +202,11 @@ public class TestDiskBasedMap {
// Test sizeEstimatorPerformance with simpleSchema
Schema schema = SchemaTestUtil.getSimpleSchema();
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
HoodieRecordConverter converter =
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName());
HoodieRecordSizeEstimator sizeEstimator =
new HoodieRecordSizeEstimator(schema);
HoodieRecord record = hoodieRecords.remove(0);
long startTime = System.currentTimeMillis();
SpillableMapUtils.computePayloadSize(record, converter);
SpillableMapUtils.computePayloadSize(record, sizeEstimator);
long timeTaken = System.currentTimeMillis() - startTime;
System.out.println("Time taken :" + timeTaken);
assertTrue(timeTaken < 100);

View File

@@ -25,7 +25,9 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
@@ -66,7 +68,8 @@ public class TestExternalSpillableMap {
String payloadClazz = HoodieAvroPayload.class.getName();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
new HoodieRecordConverter(schema, payloadClazz),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
@@ -88,7 +91,8 @@ public class TestExternalSpillableMap {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
new HoodieRecordConverter(schema, payloadClazz),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
@@ -126,7 +130,8 @@ public class TestExternalSpillableMap {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
new HoodieRecordConverter(schema, payloadClazz),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
// insert a bunch of records so that values spill to disk too
@@ -181,7 +186,8 @@ public class TestExternalSpillableMap {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, FAILURE_OUTPUT_PATH, new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
new HoodieRecordConverter(schema, payloadClazz),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
@@ -200,7 +206,8 @@ public class TestExternalSpillableMap {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
new HoodieRecordConverter(schema, payloadClazz),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk
@@ -253,7 +260,8 @@ public class TestExternalSpillableMap {
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>(16L, BASE_OUTPUT_PATH, new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
new HoodieRecordConverter(schema, payloadClazz),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); //16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk