1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -89,14 +89,13 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, unitNumber, fileName)));
assertTrue(newPath.toString().equals(
this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName)));
}
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
@@ -105,8 +104,7 @@ public class TestCopyOnWriteTable {
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
// Prepare the AvroParquetIO
String schemaStr = IOUtils
.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
}
@@ -122,28 +120,27 @@ public class TestCopyOnWriteTable {
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1));
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2));
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
// We should have a parquet file generated (TODO: better control # files after we revise AvroParquetIO)
// We should have a parquet file generated (TODO: better control # files after we revise
// AvroParquetIO)
File parquetFile = null;
for (File file : new File(this.basePath + partitionPath).listFiles()) {
if (file.getName().endsWith(".parquet")) {
@@ -155,18 +152,17 @@ public class TestCopyOnWriteTable {
// Read out the bloom filter and make sure filter can answer record exist or not
Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
BloomFilter filter = ParquetUtils
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath);
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath);
for (HoodieRecord record : records) {
assertTrue(filter.mightContain(record.getRecordKey()));
}
// Create a commit file
new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
new File(
this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + FSUtils.getCommitTime(parquetFile.getName())
+ ".commit").createNewFile();
// Read the parquet file, check the record content
List<GenericRecord> fileRecords = ParquetUtils
.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath);
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath);
GenericRecord newRecord;
int index = 0;
for (GenericRecord record : fileRecords) {
@@ -175,13 +171,12 @@ public class TestCopyOnWriteTable {
}
// We update the 1st record & add a new record
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1);
HoodieRecord updatedRecord1 = new HoodieRecord(
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()),
updateRowChanges1);
updatedRecord1.setCurrentLocation(
new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName())));
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
updatedRecord1.setCurrentLocation(new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName())));
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord insertedRecord1 = new HoodieRecord(
@@ -201,10 +196,9 @@ public class TestCopyOnWriteTable {
File updatedParquetFile = null;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet")) {
if (FSUtils.getFileId(file.getName())
.equals(FSUtils.getFileId(parquetFile.getName())) &&
HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
if (FSUtils.getFileId(file.getName()).equals(FSUtils.getFileId(parquetFile.getName()))
&& HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
updatedParquetFile = file;
break;
}
@@ -213,8 +207,8 @@ public class TestCopyOnWriteTable {
assertTrue(updatedParquetFile != null);
// Check whether the record has been updated
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
BloomFilter updatedFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath);
BloomFilter updatedFilter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(),
updatedParquetFilePath);
for (HoodieRecord record : records) {
// No change to the _row_key
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
@@ -223,8 +217,7 @@ public class TestCopyOnWriteTable {
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
records.add(insertedRecord1);// add this so it can further check below
ParquetReader updatedReader = ParquetReader
.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
index = 0;
while ((newRecord = (GenericRecord) updatedReader.read()) != null) {
assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey()));
@@ -246,13 +239,9 @@ public class TestCopyOnWriteTable {
List<HoodieRecord> records = new ArrayList<>();
for (int i = 0; i < n; i++) {
String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}",
UUID.randomUUID().toString(),
time,
i);
UUID.randomUUID().toString(), time, i);
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records.add(new HoodieRecord(
new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
rowChange));
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
}
return records;
}
@@ -261,31 +250,28 @@ public class TestCopyOnWriteTable {
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withWriteStatusClass(MetadataMergeWriteStatus.class).build();
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class)
.build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1));
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
rowChange2));
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
List<WriteStatus> writeStatuses = HoodieClientTestUtils
@@ -293,7 +279,8 @@ public class TestCopyOnWriteTable {
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus
.mergeMetadataForWriteStatuses(writeStatuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * 3
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this
// should be 2 * 3
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
@@ -314,26 +301,19 @@ public class TestCopyOnWriteTable {
List<WriteStatus> statuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
WriteStatus status = statuses.get(0);
Path partialFile = new Path(String.format("%s/%s/%s",
basePath,
status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))
);
Path partialFile = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId())));
assertTrue(fs.exists(partialFile));
// When we retry
records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
statuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
status = statuses.get(0);
Path retriedFIle = new Path(String.format("%s/%s/%s",
basePath,
status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))
);
Path retriedFIle = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId())));
assertTrue(fs.exists(retriedFIle));
assertFalse(fs.exists(partialFile));
}
@@ -371,8 +351,7 @@ public class TestCopyOnWriteTable {
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
// Insert new records
returnedStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
assertEquals(3, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
@@ -389,8 +368,8 @@ public class TestCopyOnWriteTable {
@Test
public void testFileSizeUpsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
.parquetPageSize(64 * 1024).build()).build();
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024)
.build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
@@ -398,12 +377,11 @@ public class TestCopyOnWriteTable {
List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB
for (int i = 0; i < 2000; i++) {
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString()
+ "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
String recordStr =
"{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i
+ "}";
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records
.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()),
rowChange));
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
}
// Insert new records
@@ -412,39 +390,30 @@ public class TestCopyOnWriteTable {
// Check the updated file
int counts = 0;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName())
.equals(commitTime)) {
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(commitTime)) {
System.out.println(file.getName() + "-" + file.length());
counts++;
}
}
assertEquals(
"If the number of records are more than 1150, then there should be a new file", 3,
counts);
assertEquals("If the number of records are more than 1150, then there should be a new file", 3, counts);
}
private List<HoodieCopyOnWriteTable.InsertBucket> testUpsertPartitioner(int smallFileSize,
int numInserts,
int numUpdates,
int fileSize,
boolean autoSplitInserts) throws Exception {
final String TEST_PARTITION_PATH = "2016/09/26";
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.compactionSmallFileSize(smallFileSize).insertSplitSize(100)
.autoTuneInsertSplits(autoSplitInserts).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build())
.build();
private List<HoodieCopyOnWriteTable.InsertBucket> testUpsertPartitioner(int smallFileSize, int numInserts,
int numUpdates, int fileSize, boolean autoSplitInserts) throws Exception {
final String testPartitionPath = "2016/09/26";
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize).insertSplitSize(100)
.autoTuneInsertSplits(autoSplitInserts).build()).withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(
new String[]{TEST_PARTITION_PATH});
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec : updateRecords) {
@@ -454,8 +423,8 @@ public class TestCopyOnWriteTable {
records.addAll(insertRecords);
records.addAll(updateRecords);
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner)
table.getUpsertPartitioner(profile);
HoodieCopyOnWriteTable.UpsertPartitioner partitioner =
(HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile);
assertEquals("Should have 3 partitions", 3, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
@@ -464,40 +433,35 @@ public class TestCopyOnWriteTable {
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Update record should have gone to the 1 update partiton", 0,
partitioner.getPartition(new Tuple2<>(updateRecords.get(0).getKey(),
Option.apply(updateRecords.get(0).getCurrentLocation()))));
return partitioner.getInsertBuckets(TEST_PARTITION_PATH);
assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition(
new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation()))));
return partitioner.getInsertBuckets(testPartitionPath);
}
@Test
public void testUpsertPartitioner() throws Exception {
// Inserts + Updates... Check all updates go together & inserts subsplit
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(0, 200, 100,
1024, false);
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(0, 200, 100, 1024, false);
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}
@Test
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding smallest file
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(1000 * 1024,
400, 100, 800 * 1024, false);
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding
// smallest file
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = testUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024,
false);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0,
insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight,
0.01);
assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, 0.01);
// Now with insert split size auto tuned
insertBuckets = testUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, true);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0,
insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400,
insertBuckets.get(0).weight, 0.01);
assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, insertBuckets.get(0).weight, 0.01);
}
@After