1
0

[HUDI-499] Allow update partition path with GLOBAL_BLOOM (#1187)

* Handle partition path update by deleting a record from the old partition and
  insert into the new one
* Add a new configuration "hoodie.bloom.index.update.partition.path" to
  enable the behavior
* Add a new unit test case for global bloom index
This commit is contained in:
Raymond Xu
2020-02-05 09:33:33 -08:00
committed by GitHub
parent 46842f4e92
commit c1516df8ac
4 changed files with 131 additions and 3 deletions

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.index.bloom;
import org.apache.hudi.HoodieClientTestHarness;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
@@ -28,6 +29,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
@@ -56,6 +58,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
@@ -297,6 +300,92 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
}
}
@Test
public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build())
.build();
HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
// Create the original partition, and put a record, along with the meta file
// "2016/01/31": 1 file (1_0_20160131101010.parquet)
new File(basePath + "/2016/01/31").mkdirs();
new File(basePath + "/2016/01/31/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
// this record will be saved in table and will be tagged to an empty record
TestRawTripPayload originalPayload =
new TestRawTripPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord originalRecord =
new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()),
originalPayload);
/*
This record has the same record key as originalRecord but different time so different partition
Because GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
globalBloomIndex should
- tag the original partition of the originalRecord to an empty record for deletion, and
- tag the new partition of the incomingRecord
*/
TestRawTripPayload incomingPayload =
new TestRawTripPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord incomingRecord =
new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()),
incomingPayload);
/*
This record has the same record key as originalRecord and the same partition
Though GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
globalBloomIndex should just tag the original partition
*/
TestRawTripPayload incomingPayloadSamePartition =
new TestRawTripPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}");
HoodieRecord incomingRecordSamePartition =
new HoodieRecord(
new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
incomingPayloadSamePartition);
HoodieClientTestUtils
.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(originalRecord), schema, null, false);
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
// test against incoming record with a different partition
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, jsc, table);
assertEquals(2, taggedRecordRDD.count());
for (HoodieRecord record : taggedRecordRDD.collect()) {
switch (record.getPartitionPath()) {
case "2016/01/31":
assertEquals("000", record.getRecordKey());
assertTrue(record.getData() instanceof EmptyHoodieRecordPayload);
break;
case "2016/02/31":
assertEquals("000", record.getRecordKey());
assertEquals(incomingPayload.getJsonData(), ((TestRawTripPayload) record.getData()).getJsonData());
break;
default:
fail(String.format("Should not get partition path: %s", record.getPartitionPath()));
}
}
// test against incoming record with the same partition
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
.parallelize(Collections.singletonList(incomingRecordSamePartition));
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, jsc, table);
assertEquals(1, taggedRecordRDDSamePartition.count());
HoodieRecord record = taggedRecordRDDSamePartition.first();
assertEquals("000", record.getRecordKey());
assertEquals("2016/01/31", record.getPartitionPath());
assertEquals(incomingPayloadSamePartition.getJsonData(), ((TestRawTripPayload) record.getData()).getJsonData());
}
// convert list to map to avoid sorting order dependencies
private Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();