[HUDI-1083] Optimization in determining insert bucket location for a given key (#1868)
- To determine insert bucket location for a given key, hudi walks through all insert buckets with O(N) cost, while this patch adds an optimization to make it O(logN).
This commit is contained in:
@@ -180,7 +180,7 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
// Inserts + Updates... Check all updates go together & inserts subsplit
|
||||
UpsertPartitioner partitioner = getUpsertPartitioner(0, 200, 100, 1024, testPartitionPath, false);
|
||||
List<InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
assertEquals(2, insertBuckets.size(), "Total of 2 insert buckets");
|
||||
}
|
||||
|
||||
@@ -201,20 +201,18 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
|
||||
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(insertRecords));
|
||||
UpsertPartitioner partitioner = new UpsertPartitioner(profile, jsc, table, config);
|
||||
List<InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
|
||||
float bucket0Weight = 0.2f;
|
||||
InsertBucket newInsertBucket0 = new InsertBucket();
|
||||
newInsertBucket0.bucketNumber = insertBuckets.get(0).bucketNumber;
|
||||
newInsertBucket0.weight = bucket0Weight;
|
||||
insertBuckets.remove(0);
|
||||
insertBuckets.add(0, newInsertBucket0);
|
||||
InsertBucketCumulativeWeightPair pair = insertBuckets.remove(0);
|
||||
pair.getKey().weight = bucket0Weight;
|
||||
pair.setValue(new Double(bucket0Weight));
|
||||
insertBuckets.add(0, pair);
|
||||
|
||||
InsertBucket newInsertBucket1 = new InsertBucket();
|
||||
newInsertBucket1.bucketNumber = insertBuckets.get(1).bucketNumber;
|
||||
newInsertBucket1.weight = 1 - bucket0Weight;
|
||||
insertBuckets.remove(1);
|
||||
insertBuckets.add(1, newInsertBucket1);
|
||||
InsertBucketCumulativeWeightPair pair1 = insertBuckets.remove(1);
|
||||
pair1.getKey().weight = 1 - bucket0Weight;
|
||||
pair1.setValue(new Double(1));
|
||||
insertBuckets.add(1, pair1);
|
||||
|
||||
Map<Integer, Integer> partition2numRecords = new HashMap<Integer, Integer>();
|
||||
for (HoodieRecord hoodieRecord: insertRecords) {
|
||||
@@ -238,13 +236,26 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
"The weight of bucket1 should be " + (1 - bucket0Weight));
|
||||
}
|
||||
|
||||
private void assertInsertBuckets(Double[] weights,
|
||||
Double[] cumulativeWeights,
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets) {
|
||||
for (int i = 0; i < weights.length; i++) {
|
||||
assertEquals(i, insertBuckets.get(i).getKey().bucketNumber,
|
||||
String.format("BucketNumber of insert bucket %d must be same as %d", i, i));
|
||||
assertEquals(weights[i], insertBuckets.get(i).getKey().weight, 0.01,
|
||||
String.format("Insert bucket %d should have weight %.1f", i, weights[i]));
|
||||
assertEquals(cumulativeWeights[i], insertBuckets.get(i).getValue(), 0.01,
|
||||
String.format("Insert bucket %d should have cumulativeWeight %.1f", i, cumulativeWeights[i]));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding
|
||||
// smallest file
|
||||
UpsertPartitioner partitioner = getUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, testPartitionPath, false);
|
||||
List<InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
|
||||
assertEquals(3, partitioner.numPartitions(), "Should have 3 partitions");
|
||||
assertEquals(BucketType.UPDATE, partitioner.getBucketInfo(0).bucketType,
|
||||
@@ -254,8 +265,10 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(2).bucketType,
|
||||
"Bucket 2 is INSERT");
|
||||
assertEquals(3, insertBuckets.size(), "Total of 3 insert buckets");
|
||||
assertEquals(0, insertBuckets.get(0).bucketNumber, "First insert bucket must be same as update bucket");
|
||||
assertEquals(0.5, insertBuckets.get(0).weight, 0.01, "First insert bucket should have weight 0.5");
|
||||
|
||||
Double[] weights = { 0.5, 0.25, 0.25};
|
||||
Double[] cumulativeWeights = { 0.5, 0.75, 1.0};
|
||||
assertInsertBuckets(weights, cumulativeWeights, insertBuckets);
|
||||
|
||||
// Now with insert split size auto tuned
|
||||
partitioner = getUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, testPartitionPath, true);
|
||||
@@ -271,8 +284,10 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(3).bucketType,
|
||||
"Bucket 3 is INSERT");
|
||||
assertEquals(4, insertBuckets.size(), "Total of 4 insert buckets");
|
||||
assertEquals(0, insertBuckets.get(0).bucketNumber, "First insert bucket must be same as update bucket");
|
||||
assertEquals(200.0 / 2400, insertBuckets.get(0).weight, 0.01, "First insert bucket should have weight 0.5");
|
||||
|
||||
weights = new Double[] { 0.08, 0.31, 0.31, 0.31};
|
||||
cumulativeWeights = new Double[] { 0.08, 0.39, 0.69, 1.0};
|
||||
assertInsertBuckets(weights, cumulativeWeights, insertBuckets);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
||||
|
||||
Reference in New Issue
Block a user