[HUDI-472] Introduce configurations and new modes of sorting for bulk_insert (#1149)
* [HUDI-472] Introduce the configuration and new modes of record sorting for bulk_insert(#1149). Three sorting modes are implemented: global sort ("global_sort"), local sort inside each RDD partition ("partition_sort") and no sort ("none")
This commit is contained in:
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
|
||||
|
||||
public static JavaRDD<HoodieRecord> generateTestRecordsForBulkInsert(JavaSparkContext jsc) {
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
// RDD partition 1
|
||||
List<HoodieRecord> records1 = dataGenerator.generateInserts("0", 100);
|
||||
// RDD partition 2
|
||||
List<HoodieRecord> records2 = dataGenerator.generateInserts("0", 150);
|
||||
return jsc.parallelize(records1, 1).union(jsc.parallelize(records2, 1));
|
||||
}
|
||||
|
||||
public static Map<String, Long> generateExpectedPartitionNumRecords(JavaRDD<HoodieRecord> records) {
|
||||
return records.map(record -> record.getPartitionPath()).countByValue();
|
||||
}
|
||||
|
||||
private static JavaRDD<HoodieRecord> generateTripleTestRecordsForBulkInsert(JavaSparkContext jsc)
|
||||
throws Exception {
|
||||
return generateTestRecordsForBulkInsert(jsc).union(generateTestRecordsForBulkInsert(jsc))
|
||||
.union(generateTestRecordsForBulkInsert(jsc));
|
||||
}
|
||||
|
||||
private static Stream<Arguments> configParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{BulkInsertInternalPartitionerFactory.BulkInsertSortMode.GLOBAL_SORT, true, true},
|
||||
{BulkInsertInternalPartitionerFactory.BulkInsertSortMode.PARTITION_SORT, false, true},
|
||||
{BulkInsertInternalPartitionerFactory.BulkInsertSortMode.NONE, false, false}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
private void verifyRecordAscendingOrder(List<HoodieRecord> records) {
|
||||
List<HoodieRecord> expectedRecords = new ArrayList<>(records);
|
||||
Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey())));
|
||||
assertEquals(expectedRecords, records);
|
||||
}
|
||||
|
||||
private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner,
|
||||
JavaRDD<HoodieRecord> records,
|
||||
boolean isGloballySorted, boolean isLocallySorted,
|
||||
Map<String, Long> expectedPartitionNumRecords) {
|
||||
int numPartitions = 2;
|
||||
JavaRDD<HoodieRecord> actualRecords = partitioner.repartitionRecords(records, numPartitions);
|
||||
assertEquals(numPartitions, actualRecords.getNumPartitions());
|
||||
List<HoodieRecord> collectedActualRecords = actualRecords.collect();
|
||||
if (isGloballySorted) {
|
||||
// Verify global order
|
||||
verifyRecordAscendingOrder(collectedActualRecords);
|
||||
} else if (isLocallySorted) {
|
||||
// Verify local order
|
||||
actualRecords.mapPartitions(partition -> {
|
||||
List<HoodieRecord> partitionRecords = new ArrayList<>();
|
||||
partition.forEachRemaining(partitionRecords::add);
|
||||
verifyRecordAscendingOrder(partitionRecords);
|
||||
return Collections.emptyList().iterator();
|
||||
}).collect();
|
||||
}
|
||||
|
||||
// Verify number of records per partition path
|
||||
Map<String, Long> actualPartitionNumRecords = new HashMap<>();
|
||||
for (HoodieRecord record : collectedActualRecords) {
|
||||
String partitionPath = record.getPartitionPath();
|
||||
actualPartitionNumRecords.put(partitionPath,
|
||||
actualPartitionNumRecords.getOrDefault(partitionPath, 0L) + 1);
|
||||
}
|
||||
assertEquals(expectedPartitionNumRecords, actualPartitionNumRecords);
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = "[{index}] {0}")
|
||||
@MethodSource("configParams")
|
||||
public void testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.BulkInsertSortMode sortMode,
|
||||
boolean isGloballySorted, boolean isLocallySorted)
|
||||
throws Exception {
|
||||
JavaRDD<HoodieRecord> records1 = generateTestRecordsForBulkInsert(jsc);
|
||||
JavaRDD<HoodieRecord> records2 = generateTripleTestRecordsForBulkInsert(jsc);
|
||||
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode),
|
||||
records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1));
|
||||
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode),
|
||||
records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2));
|
||||
}
|
||||
}
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
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.HoodieStorageConfig;
|
||||
@@ -52,16 +53,23 @@ import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateExpectedPartitionNumRecords;
|
||||
import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateTestRecordsForBulkInsert;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
@@ -290,6 +298,21 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
|
||||
}
|
||||
|
||||
private void verifyStatusResult(List<WriteStatus> statuses, Map<String, Long> expectedPartitionNumRecords) {
|
||||
Map<String, Long> actualPartitionNumRecords = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < statuses.size(); i++) {
|
||||
WriteStatus writeStatus = statuses.get(i);
|
||||
String partitionPath = writeStatus.getPartitionPath();
|
||||
actualPartitionNumRecords.put(
|
||||
partitionPath,
|
||||
actualPartitionNumRecords.getOrDefault(partitionPath, 0L) + writeStatus.getTotalRecords());
|
||||
assertEquals(0, writeStatus.getFailedRecords().size());
|
||||
}
|
||||
|
||||
assertEquals(expectedPartitionNumRecords, actualPartitionNumRecords);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
@@ -312,12 +335,10 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
// TODO: check the actual files and make sure 11 records, total were written.
|
||||
assertEquals(2, returnedStatuses.size());
|
||||
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
|
||||
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
|
||||
assertEquals(10, returnedStatuses.get(0).getTotalRecords());
|
||||
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
|
||||
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
|
||||
assertEquals(1, returnedStatuses.get(1).getTotalRecords());
|
||||
Map<String, Long> expectedPartitionNumRecords = new HashMap<>();
|
||||
expectedPartitionNumRecords.put("2016/01/31", 10L);
|
||||
expectedPartitionNumRecords.put("2016/02/01", 1L);
|
||||
verifyStatusResult(returnedStatuses, expectedPartitionNumRecords);
|
||||
|
||||
// Case 2:
|
||||
// 1 record for partition 1, 5 record for partition 2, 1 records for partition 3.
|
||||
@@ -334,14 +355,11 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
|
||||
assertEquals(3, returnedStatuses.size());
|
||||
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
|
||||
assertEquals(1, returnedStatuses.get(0).getTotalRecords());
|
||||
|
||||
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
|
||||
assertEquals(5, returnedStatuses.get(1).getTotalRecords());
|
||||
|
||||
assertEquals("2016/02/02", returnedStatuses.get(2).getPartitionPath());
|
||||
assertEquals(1, returnedStatuses.get(2).getTotalRecords());
|
||||
expectedPartitionNumRecords.clear();
|
||||
expectedPartitionNumRecords.put("2016/01/31", 1L);
|
||||
expectedPartitionNumRecords.put("2016/02/01", 5L);
|
||||
expectedPartitionNumRecords.put("2016/02/02", 1L);
|
||||
verifyStatusResult(returnedStatuses, expectedPartitionNumRecords);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -399,7 +417,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
metaClient.getFs().create(new Path(Paths.get(basePath, ".hoodie", "000.commit").toString())).close();
|
||||
final List<HoodieRecord> updates = dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts);
|
||||
|
||||
String partitionPath = updates.get(0).getPartitionPath();
|
||||
String partitionPath = writeStatus.getPartitionPath();
|
||||
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
|
||||
CommitActionExecutor newActionExecutor = new UpsertCommitActionExecutor(jsc, config, table,
|
||||
instantTime, jsc.parallelize(updates));
|
||||
@@ -408,4 +426,28 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
|
||||
}
|
||||
|
||||
public void testBulkInsertRecords(String bulkInsertMode) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withBulkInsertParallelism(2).withBulkInsertSortMode(bulkInsertMode).build();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieWriteClient writeClient = getHoodieWriteClient(config);
|
||||
writeClient.startCommitWithTime(instantTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
|
||||
|
||||
// Insert new records
|
||||
final JavaRDD<HoodieRecord> inputRecords = generateTestRecordsForBulkInsert(jsc);
|
||||
BulkInsertCommitActionExecutor bulkInsertExecutor = new BulkInsertCommitActionExecutor(
|
||||
jsc, config, table, instantTime, inputRecords, Option.empty());
|
||||
List<WriteStatus> returnedStatuses = bulkInsertExecutor.execute().getWriteStatuses().collect();
|
||||
verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords));
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = "[{index}] {0}")
|
||||
@ValueSource(strings = {"global_sort", "partition_sort", "none"})
|
||||
public void testBulkInsertRecordsWithGlobalSort(String bulkInsertMode) throws Exception {
|
||||
testBulkInsertRecords(bulkInsertMode);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user