Enabling hard deletes for MergeOnRead table type
This commit is contained in:
committed by
vinoth chandar
parent
345aaa31aa
commit
110df7190b
@@ -25,6 +25,7 @@ import static org.junit.Assert.fail;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
|
||||
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
@@ -43,6 +44,7 @@ import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
import com.uber.hoodie.exception.CorruptedLogFileException;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
@@ -714,10 +716,13 @@ public class HoodieLogFormatTest {
|
||||
s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList());
|
||||
|
||||
// Delete 50 keys
|
||||
List<String> deletedKeys = originalKeys.subList(0, 50);
|
||||
List<HoodieKey> deletedKeys = copyOfRecords1.stream().map(
|
||||
s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(),
|
||||
((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString())))
|
||||
.collect(Collectors.toList()).subList(0, 50);
|
||||
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102");
|
||||
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header);
|
||||
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
|
||||
writer = writer.appendBlock(deleteBlock);
|
||||
|
||||
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
|
||||
@@ -727,8 +732,19 @@ public class HoodieLogFormatTest {
|
||||
"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);
|
||||
final List<Boolean> emptyPayloads = new ArrayList<>();
|
||||
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
|
||||
assertEquals("Stream collect should return all 150 records", 150, readKeys.size());
|
||||
scanner.forEach(s -> {
|
||||
try {
|
||||
if (!s.getData().getInsertValue(schema).isPresent()) {
|
||||
emptyPayloads.add(true);
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
});
|
||||
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
|
||||
assertEquals("Stream collect should return all 50 records with empty payloads", 50, emptyPayloads.size());
|
||||
originalKeys.removeAll(deletedKeys);
|
||||
Collections.sort(originalKeys);
|
||||
Collections.sort(readKeys);
|
||||
@@ -782,8 +798,13 @@ public class HoodieLogFormatTest {
|
||||
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]), header);
|
||||
// Delete 50 keys
|
||||
List<HoodieKey> deletedKeys = copyOfRecords1.stream().map(
|
||||
s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(),
|
||||
((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString())))
|
||||
.collect(Collectors.toList()).subList(0, 50);
|
||||
|
||||
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
|
||||
writer = writer.appendBlock(deleteBlock);
|
||||
|
||||
// Attempt 1 : Write rollback block for a failed write
|
||||
@@ -839,8 +860,12 @@ public class HoodieLogFormatTest {
|
||||
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]), header);
|
||||
// Delete 50 keys
|
||||
List<HoodieKey> deletedKeys = copyOfRecords1.stream().map(
|
||||
s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(),
|
||||
((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString())))
|
||||
.collect(Collectors.toList()).subList(0, 50);
|
||||
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
|
||||
writer = writer.appendBlock(deleteBlock);
|
||||
|
||||
// Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write
|
||||
@@ -921,8 +946,12 @@ public class HoodieLogFormatTest {
|
||||
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]), header);
|
||||
// Delete 50 keys
|
||||
List<HoodieKey> deletedKeys = copyOfRecords1.stream().map(
|
||||
s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(),
|
||||
((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString())))
|
||||
.collect(Collectors.toList()).subList(0, 50);
|
||||
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header);
|
||||
writer = writer.appendBlock(deleteBlock);
|
||||
|
||||
// Write 1 rollback block for a failed write
|
||||
|
||||
Reference in New Issue
Block a user