1
0

1) Separated rollback as a table operation 2) Implement rollback for MOR

This commit is contained in:
Nishith Agarwal
2017-08-17 17:31:55 -07:00
committed by vinoth chandar
parent e1fe3ab937
commit c7d63a7622
28 changed files with 1200 additions and 260 deletions

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.common.model;
import com.google.common.collect.Lists;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.table.HoodieTableConfig;
@@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
@@ -249,6 +250,8 @@ public class HoodieTestUtils {
.overBaseCommit(location.getCommitTime())
.withFs(fs).build();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, location.getCommitTime());
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
@@ -260,7 +263,7 @@ public class HoodieTestUtils {
} catch (IOException e) {
return null;
}
}).collect(Collectors.toList()), schema));
}).collect(Collectors.toList()), schema, metadata));
logWriter.close();
} catch (Exception e) {
fail(e.toString());

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.common.table.log;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -24,16 +25,13 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -49,7 +47,12 @@ import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
@@ -107,8 +110,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
assertTrue("We just wrote a block - size should be > 0", size > 0);
@@ -124,8 +129,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
// Write out a block
writer = writer.appendBlock(dataBlock);
// Get the size of the block
@@ -138,7 +145,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
assertEquals("This should be a new log file and hence size should be 0", 0,
writer.getCurrentSize());
@@ -152,8 +159,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -163,7 +172,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
@@ -178,7 +187,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2);
@@ -202,8 +211,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
// do not close this writer - this simulates a data note appending to a log dying without closing the file
@@ -214,7 +225,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
@@ -232,12 +243,14 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("We wrote a block, we should be able to read it", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK,
@@ -257,8 +270,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -267,7 +282,7 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -277,11 +292,11 @@ public class HoodieLogFormatTest {
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
@@ -312,8 +327,10 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -324,20 +341,24 @@ public class HoodieLogFormatTest {
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.writeInt(1000);
// Write out some metadata
// TODO : test for failure to write metadata - NA ?
outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// First round of reads - we should be able to read the first block and then EOF
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should have corrupted block next", reader.hasNext());
HoodieLogBlock block = reader.next();
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
assertEquals("", "something-random", new String(block.getBytes()));
HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block;
assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes()));
assertFalse("There should be no more block left", reader.hasNext());
// Simulate another failure back to back
@@ -346,7 +367,10 @@ public class HoodieLogFormatTest {
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.writeInt(1000);
// Write out some metadata
// TODO : test for failure to write metadata - NA ?
outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
outputStream.write("something-else-random".getBytes());
outputStream.flush();
outputStream.close();
@@ -356,12 +380,12 @@ public class HoodieLogFormatTest {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema());
dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema(), metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
// Second round of reads - we should be able to read the first and last block
reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), true);
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should get the 1st corrupted block next", reader.hasNext());
@@ -370,7 +394,8 @@ public class HoodieLogFormatTest {
block = reader.next();
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
assertEquals("", "something-else-random", new String(block.getBytes()));
corruptBlock = (HoodieCorruptBlock) block;
assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes()));
assertTrue("We should get the last block next", reader.hasNext());
reader.next();
assertFalse("We should have no more blocks left", reader.hasNext());
@@ -388,12 +413,15 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -402,8 +430,7 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -428,22 +455,27 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
// Rollback the last write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
// Write 3
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3, schema);
dataBlock = new HoodieAvroDataBlock(records3, schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -452,9 +484,8 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
assertEquals("We still would read 300 records, but only 200 of them are valid", 300,
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We only read 200 records, since 200 of them are valid", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -479,7 +510,11 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -491,13 +526,16 @@ public class HoodieLogFormatTest {
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
// Write out some metadata
// TODO : test for failure to write metadata - NA ?
outputStream.write(HoodieLogBlock.getLogMetadataBytes(metadata));
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// Rollback the last write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK);
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
@@ -505,7 +543,7 @@ public class HoodieLogFormatTest {
// Write 3
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3, schema);
dataBlock = new HoodieAvroDataBlock(records3, schema, metadata);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -514,8 +552,7 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We would read 200 records", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -531,7 +568,7 @@ public class HoodieLogFormatTest {
}
@Test
public void testAvroLogRecordReaderWithDelete()
public void testAvroLogRecordReaderWithDeleteAndRollback()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
@@ -541,12 +578,16 @@ public class HoodieLogFormatTest {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
records1.addAll(records2);
@@ -557,7 +598,8 @@ public class HoodieLogFormatTest {
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]));
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
writer = writer.appendBlock(deleteBlock);
List<String> allLogFiles = FSUtils
@@ -565,17 +607,170 @@ public class HoodieLogFormatTest {
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
assertEquals("We still would read 200 records", 200,
scanner.getTotalLogRecords());
List<String> readKeys = new ArrayList<>(200);
final List<String> readKeys = new ArrayList<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 150, readKeys.size());
assertEquals("Stream collect should return all 150 records", 150, readKeys.size());
originalKeys.removeAll(deletedKeys);
Collections.sort(originalKeys);
Collections.sort(readKeys);
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys,
assertEquals("CompositeAvroLogReader should return 150 records from 2 versions", originalKeys,
readKeys);
// Rollback the last block
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles, schema, "100");
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size());
}
@Test
public void testAvroLogRecordReaderWithFailedRollbacks()
throws IOException, URISyntaxException, InterruptedException {
// Write a Data block and Delete block with same InstantTime (written in same batch)
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema, metadata);
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
writer = writer.appendBlock(deleteBlock);
// Attemp 1 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
try {
writer = writer.appendBlock(commandBlock);
// Say job failed, retry writing 2 rollback in the next rollback(..) attempt
throw new Exception("simulating failure");
} catch(Exception e) {
// it's okay
}
// Attempt 2 : Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
writer = writer.appendBlock(commandBlock);
writer = writer.appendBlock(commandBlock);
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, allLogFiles, schema, "100");
assertEquals("We would read 100 records", 100,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());
}
@Test
public void testAvroLogRecordReaderWithInsertDeleteAndRollback()
throws IOException, URISyntaxException, InterruptedException {
// Write a Data block and Delete block with same InstantTime (written in same batch)
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "100");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), metadata);
writer = writer.appendBlock(deleteBlock);
// Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
writer = writer.appendBlock(commandBlock);
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, allLogFiles, schema, "100");
assertEquals("We would read 0 records", 0,
scanner.getTotalLogRecords());
}
@Test
public void testAvroLogRecordReaderWithInvalidRollback() throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, "101");
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
schema, metadata);
writer = writer.appendBlock(dataBlock);
// Write invalid rollback for a failed write (possible for in-flight commits)
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata);
writer = writer.appendBlock(commandBlock);
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, allLogFiles, schema, "100");
assertEquals("We still would read 100 records", 100,
scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 150 records", 100, readKeys.size());
}
}