[HUDI-2245] BucketAssigner generates the fileId evenly to avoid data skew (#3362)
This commit is contained in:
@@ -79,6 +79,35 @@ public class TestBucketAssigner {
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that the file ids generated by the task can finally shuffled to itself.
|
||||
*/
|
||||
@Test
|
||||
void testSmallFilesOfThisTask() {
|
||||
MockBucketAssigner mockBucketAssigner1 = new MockBucketAssigner(context, writeConfig);
|
||||
String fileId1 = mockBucketAssigner1.createFileIdOfThisTask();
|
||||
SmallFile smallFile1 = new SmallFile();
|
||||
smallFile1.location = new HoodieRecordLocation("t0", fileId1);
|
||||
smallFile1.sizeBytes = 123;
|
||||
List<SmallFile> smallFiles1 = mockBucketAssigner1.smallFilesOfThisTask(Collections.singletonList(smallFile1));
|
||||
assertThat(smallFiles1.size(), is(1));
|
||||
|
||||
// modify the parallelism and test again
|
||||
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(123, 200, context, writeConfig, Collections.emptyMap());
|
||||
String fileId2 = mockBucketAssigner2.createFileIdOfThisTask();
|
||||
SmallFile smallFile2 = new SmallFile();
|
||||
smallFile2.location = new HoodieRecordLocation("t0", fileId2);
|
||||
smallFile2.sizeBytes = 123;
|
||||
|
||||
String fileId3 = mockBucketAssigner2.createFileIdOfThisTask();
|
||||
SmallFile smallFile3 = new SmallFile();
|
||||
smallFile3.location = new HoodieRecordLocation("t0", fileId3);
|
||||
smallFile3.sizeBytes = 456;
|
||||
|
||||
List<SmallFile> smallFiles2 = mockBucketAssigner1.smallFilesOfThisTask(Arrays.asList(smallFile2, smallFile3));
|
||||
assertThat(smallFiles2.size(), is(2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddUpdate() {
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
|
||||
@@ -200,11 +229,11 @@ public class TestBucketAssigner {
|
||||
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(0, 2, context, writeConfig, smallFilesMap);
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
@@ -214,11 +243,11 @@ public class TestBucketAssigner {
|
||||
|
||||
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(1, 2, context, writeConfig, smallFilesMap);
|
||||
BucketInfo bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner2.addInsert("par1");
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo2, "par3", BucketType.INSERT);
|
||||
@@ -292,34 +321,34 @@ public class TestBucketAssigner {
|
||||
mockBucketAssigner.addUpdate("par1", "f0");
|
||||
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
mockBucketAssigner.addUpdate("par1", "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
|
||||
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(1, 2, context, writeConfig, smallFilesMap);
|
||||
mockBucketAssigner2.addUpdate("par1", "f0");
|
||||
|
||||
BucketInfo bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner2.addInsert("par1");
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner2.addUpdate("par1", "f2");
|
||||
|
||||
mockBucketAssigner2.addInsert("par1");
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -425,7 +454,7 @@ public class TestBucketAssigner {
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
Map<String, List<SmallFile>> smallFilesMap) {
|
||||
super(taskID, numTasks, new MockWriteProfile(config, context, smallFilesMap), config);
|
||||
super(taskID, 1024, numTasks, new MockWriteProfile(config, context, smallFilesMap), config);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user