1
0

[HUDI-770] Organize upsert/insert API implementation under a single package (#1495)

This commit is contained in:
Balaji Varadarajan
2020-04-12 23:11:00 -07:00
committed by GitHub
parent 447ba3bae6
commit 17bf930342
39 changed files with 2703 additions and 859 deletions

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.HoodieClientTestHarness;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
@@ -222,7 +223,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
*
* @param records List of Hoodie records
*/
void assertNodupesWithinPartition(List<HoodieRecord> records) {
void assertNodupesWithinPartition(List<HoodieRecord<TestRawTripPayload>> records) {
Map<String, Set<String>> partitionToKeys = new HashMap<>();
for (HoodieRecord r : records) {
String key = r.getRecordKey();

View File

@@ -18,8 +18,10 @@
package org.apache.hudi.client;
import java.util.HashSet;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -50,6 +52,7 @@ import org.apache.hudi.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.action.commit.WriteHelper;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -195,7 +198,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
String recordKey = UUID.randomUUID().toString();
HoodieKey keyOne = new HoodieKey(recordKey, "2018-01-01");
HoodieRecord recordOne =
HoodieRecord<TestRawTripPayload> recordOne =
new HoodieRecord(keyOne, HoodieTestDataGenerator.generateRandomValue(keyOne, newCommitTime));
HoodieKey keyTwo = new HoodieKey(recordKey, "2018-02-01");
@@ -206,42 +209,51 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieRecord recordThree =
new HoodieRecord(keyTwo, HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime));
JavaRDD<HoodieRecord> records = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
JavaRDD<HoodieRecord<TestRawTripPayload>> records =
jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
// dedup should be done based on recordKey only
HoodieWriteClient clientWithDummyGlobalIndex = getWriteClientWithDummyIndex(true);
List<HoodieRecord> dedupedRecs = clientWithDummyGlobalIndex.deduplicateRecords(records, 1).collect();
// Global dedup should be done based on recordKey only
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(true);
List<HoodieRecord<TestRawTripPayload>> dedupedRecs = WriteHelper.deduplicateRecords(records, index, 1).collect();
assertEquals(1, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);
// dedup should be done based on both recordKey and partitionPath
HoodieWriteClient clientWithDummyNonGlobalIndex = getWriteClientWithDummyIndex(false);
dedupedRecs = clientWithDummyNonGlobalIndex.deduplicateRecords(records, 1).collect();
// non-Global dedup should be done based on both recordKey and partitionPath
index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(false);
dedupedRecs = WriteHelper.deduplicateRecords(records, index, 1).collect();
assertEquals(2, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);
// Perform write-action and check
JavaRDD<HoodieRecord> recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
try (HoodieWriteClient client = getHoodieWriteClient(getConfigBuilder().combineInput(true, true).build(), false);) {
client.startCommitWithTime(newCommitTime);
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
List<WriteStatus> statuses = writeFn.apply(client, recordList, newCommitTime).collect();
assertNoWriteErrors(statuses);
assertEquals(2, statuses.size());
assertNodupesWithinPartition(statuses.stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
assertNodupesInPartition(statuses.stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
.collect(Collectors.toList()));
}
}
/**
* Build a test Hoodie WriteClient with dummy index to configure isGlobal flag.
* Assert that there is no duplicate key at the partition level.
*
* @param isGlobal Flag to control HoodieIndex.isGlobal
* @return Hoodie Write Client
* @throws Exception in case of error
* @param records List of Hoodie records
*/
private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) {
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(isGlobal);
return getHoodieWriteClient(getConfigBuilder().build(), false, index);
void assertNodupesInPartition(List<HoodieRecord> records) {
Map<String, Set<String>> partitionToKeys = new HashMap<>();
for (HoodieRecord r : records) {
String key = r.getRecordKey();
String partitionPath = r.getPartitionPath();
if (!partitionToKeys.containsKey(partitionPath)) {
partitionToKeys.put(partitionPath, new HashSet<>());
}
assertFalse("key " + key + " is duplicate within partition " + partitionPath, partitionToKeys.get(partitionPath).contains(key));
partitionToKeys.get(partitionPath).add(key);
}
}
/**

View File

@@ -61,6 +61,8 @@ import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.action.deltacommit.DeltaCommitActionExecutor;
import org.apache.hudi.table.action.deltacommit.DeleteDeltaCommitActionExecutor;
import org.apache.spark.api.java.JavaRDD;
import org.junit.After;
import org.junit.Assert;
@@ -1346,9 +1348,11 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
JavaRDD<HoodieRecord> deleteRDD = jsc.parallelize(fewRecordsForDelete, 1);
// initialize partitioner
hoodieTable.getUpsertPartitioner(new WorkloadProfile(deleteRDD), jsc);
DeltaCommitActionExecutor actionExecutor = new DeleteDeltaCommitActionExecutor(jsc, cfg, hoodieTable,
newDeleteTime, deleteRDD);
actionExecutor.getUpsertPartitioner(new WorkloadProfile(deleteRDD));
final List<List<WriteStatus>> deleteStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
return hoodieTable.handleUpdate(newDeleteTime, partitionPath, fileId, fewRecordsForDelete.iterator());
return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator());
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
// Verify there are errors because records are from multiple partitions (but handleUpdate is invoked for

View File

@@ -16,34 +16,31 @@
* limitations under the License.
*/
package org.apache.hudi.table;
package org.apache.hudi.table.action.commit;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.HoodieClientTestHarness;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.hadoop.HoodieHiveUtil;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.io.HoodieCreateHandle;
import org.apache.hudi.table.HoodieCopyOnWriteTable.UpsertPartitioner;
import org.apache.hudi.table.HoodieCopyOnWriteTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
@@ -68,20 +65,18 @@ import java.util.List;
import java.util.Map;
import java.util.UUID;
import scala.Tuple2;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestCopyOnWriteTable extends HoodieClientTestHarness {
public class TestCopyOnWriteActionExecutor extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteTable.class);
private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteActionExecutor.class);
@Before
public void setUp() throws Exception {
initSparkContexts("TestCopyOnWriteTable");
initSparkContexts("TestCopyOnWriteActionExecutor");
initPath();
initMetaClient();
initTestDataGenerator();
@@ -179,7 +174,8 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
GenericRecord newRecord;
int index = 0;
for (GenericRecord record : fileRecords) {
assertTrue(record.get("_row_key").toString().equals(records.get(index).getRecordKey()));
System.out.println("Got :" + record.get("_row_key").toString() + ", Exp :" + records.get(index).getRecordKey());
assertEquals(records.get(index).getRecordKey(), record.get("_row_key").toString());
index++;
}
@@ -300,8 +296,10 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
CommitActionExecutor actionExecutor = new InsertCommitActionExecutor(jsc, config, table,
firstCommitTime, jsc.parallelize(records));
List<WriteStatus> writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(firstCommitTime, FSUtils.createNewFileIdPfx(), records.iterator());
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
Map<String, String> allWriteStatusMergedMetadataMap =
@@ -326,8 +324,10 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
// Insert new records
final List<HoodieRecord> recs2 = records;
CommitActionExecutor actionExecutor = new InsertPreppedCommitActionExecutor(jsc, config, table,
instantTime, jsc.parallelize(recs2));
List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(instantTime, FSUtils.createNewFileIdPfx(), recs2.iterator());
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
// TODO: check the actual files and make sure 11 records, total were written.
@@ -347,9 +347,10 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
// Insert new records
final List<HoodieRecord> recs3 = records;
CommitActionExecutor newActionExecutor = new UpsertPreppedCommitActionExecutor(jsc, config, table,
instantTime, jsc.parallelize(recs3));
returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(instantTime, FSUtils.createNewFileIdPfx(), recs3.iterator());
return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
assertEquals(3, returnedStatuses.size());
@@ -361,7 +362,6 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
assertEquals("2016/02/02", returnedStatuses.get(2).getPartitionPath());
assertEquals(1, returnedStatuses.get(2).getTotalRecords());
}
@Test
@@ -382,8 +382,10 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
}
// Insert new records
CommitActionExecutor actionExecutor = new UpsertCommitActionExecutor(jsc, config, table,
instantTime, jsc.parallelize(records));
jsc.parallelize(Arrays.asList(1))
.map(i -> table.handleInsert(instantTime, FSUtils.createNewFileIdPfx(), records.iterator()))
.map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()))
.map(x -> HoodieClientTestUtils.collectStatuses(x)).collect();
// Check the updated file
@@ -397,83 +399,6 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
assertEquals("If the number of records are more than 1150, then there should be a new file", 3, counts);
}
private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, int numUpdates, int fileSize,
String testPartitionPath, boolean autoSplitInserts) throws Exception {
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, testPartitionPath, "001", "file1", fileSize);
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
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) {
updateRec.unseal();
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
updateRec.seal();
}
List<HoodieRecord> records = new ArrayList<>();
records.addAll(insertRecords);
records.addAll(updateRecords);
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
HoodieCopyOnWriteTable.UpsertPartitioner partitioner =
(HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile, jsc);
assertEquals("Update record should have gone to the 1 update partition", 0, partitioner.getPartition(
new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))));
return partitioner;
}
@Test
public void testUpsertPartitioner() throws Exception {
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<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}
@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<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Should have 3 partitions", 3, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
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);
// Now with insert split size auto tuned
partitioner = getUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, testPartitionPath, true);
insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Should have 4 partitions", 4, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Bucket 3 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(3).bucketType);
assertEquals("Total of 4 insert buckets", 4, 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);
}
@Test
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
@@ -483,8 +408,10 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
String instantTime = "000";
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
CommitActionExecutor actionExecutor = new InsertCommitActionExecutor(jsc, config, table,
instantTime, jsc.parallelize(inserts));
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(instantTime, UUID.randomUUID().toString(), inserts.iterator());
return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator());
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
WriteStatus writeStatus = ws.get(0).get(0);
@@ -494,8 +421,10 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
String partitionPath = updates.get(0).getPartitionPath();
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
CommitActionExecutor newActionExecutor = new UpsertCommitActionExecutor(jsc, config, table,
instantTime, jsc.parallelize(updates));
final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleUpdate(instantTime, partitionPath, fileId, updates.iterator());
return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator());
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
}

View File

@@ -0,0 +1,148 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.commit;
import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
import java.util.List;
import org.apache.hudi.common.HoodieClientTestHarness;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieCopyOnWriteTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import scala.Tuple2;
public class TestUpsertPartitioner extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestUpsertPartitioner.class);
@Before
public void setUp() throws Exception {
initSparkContexts("TestUpsertPartitioner");
initPath();
initMetaClient();
initTestDataGenerator();
initFileSystem();
}
@After
public void tearDown() throws Exception {
cleanupSparkContexts();
cleanupMetaClient();
cleanupFileSystem();
cleanupTestDataGenerator();
}
private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, int numUpdates, int fileSize,
String testPartitionPath, boolean autoSplitInserts) throws Exception {
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, testPartitionPath, "001", "file1", fileSize);
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, jsc);
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) {
updateRec.unseal();
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
updateRec.seal();
}
List<HoodieRecord> records = new ArrayList<>();
records.addAll(insertRecords);
records.addAll(updateRecords);
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
UpsertPartitioner partitioner = new UpsertPartitioner(profile, jsc, table, config);
assertEquals("Update record should have gone to the 1 update partition", 0, partitioner.getPartition(
new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))));
return partitioner;
}
@Test
public void testUpsertPartitioner() throws Exception {
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);
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}
@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);
assertEquals("Should have 3 partitions", 3, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
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);
// Now with insert split size auto tuned
partitioner = getUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, testPartitionPath, true);
insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Should have 4 partitions", 4, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Bucket 3 is INSERT", BucketType.INSERT,
partitioner.getBucketInfo(3).bucketType);
assertEquals("Total of 4 insert buckets", 4, 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);
}
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
// Prepare the AvroParquetIO
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
}
}