1
0

Fixed HUDI-116 : Handle duplicate record keys across partitions

- Join based on HoodieKey and not RecordKey during tagging
 - Unit tests changed to run with duplicate keys
 - Special casing GlobalBloom to still join by recordkey
This commit is contained in:
Vinoth Chandar
2019-05-21 18:59:10 -07:00
committed by vinoth chandar
parent f120427607
commit 4074c5eb23
5 changed files with 82 additions and 90 deletions

View File

@@ -127,43 +127,6 @@ public class TestHoodieBloomIndex {
return config;
}
@Test
public void testLoadUUIDsInMemory() throws IOException {
// Create one RDD of hoodie record
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()),
rowChange4);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
// Load to memory
Map<String, Iterable<String>> map = recordRDD.mapToPair(
record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())).groupByKey().collectAsMap();
assertEquals(map.size(), 2);
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
assertEquals(list1.size(), 3);
assertEquals(list2.size(), 1);
}
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = makeConfig();
@@ -354,14 +317,14 @@ public class TestHoodieBloomIndex {
String rowKey1 = UUID.randomUUID().toString();
String rowKey2 = UUID.randomUUID().toString();
String rowKey3 = UUID.randomUUID().toString();
String rowKey4 = UUID.randomUUID().toString();
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\","
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"" + rowKey4 + "\","
// place same row key under a different partition.
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\","
+ "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
@@ -408,13 +371,15 @@ public class TestHoodieBloomIndex {
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getRecordKey().equals(rowKey1)) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
if (record.getPartitionPath().equals("2015/01/31")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
} else {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
}
} else if (record.getRecordKey().equals(rowKey2)) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
} else if (record.getRecordKey().equals(rowKey3)) {
assertTrue(!record.isCurrentLocationKnown());
} else if (record.getRecordKey().equals(rowKey4)) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
}
}
}
@@ -429,7 +394,8 @@ public class TestHoodieBloomIndex {
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
// record key same as recordStr2
String recordStr4 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
@@ -439,7 +405,6 @@ public class TestHoodieBloomIndex {
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
HoodieRecord record3 = new HoodieRecord(key3, rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
@@ -481,13 +446,13 @@ public class TestHoodieBloomIndex {
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path2 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
if (record._1.getPartitionPath().equals("2015/01/31")) {
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path2.getName()));
} else {
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
}
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(!record._2.isPresent());
} else if (record._1.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
Path path3 = new Path(record._2.get());
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path3.getName()));
}
}
}