[HUDI-1104] Adding support for UserDefinedPartitioners and SortModes to BulkInsert with Rows (#3149)
This commit is contained in:
committed by
GitHub
parent
55ecbc662e
commit
ea9e5d0e8b
@@ -0,0 +1,42 @@
|
|||||||
|
/*
|
||||||
|
* 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.config;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.config.HoodieConfig;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configs/params used for internal purposes.
|
||||||
|
*/
|
||||||
|
public class HoodieInternalConfig extends HoodieConfig {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 0L;
|
||||||
|
|
||||||
|
public static final String BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED = "hoodie.bulkinsert.are.partitioner.records.sorted";
|
||||||
|
public static final Boolean DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED = false;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns if partition records are sorted or not.
|
||||||
|
* @param propertyValue value for property BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED.
|
||||||
|
* @return the property value.
|
||||||
|
*/
|
||||||
|
public static Boolean getBulkInsertIsPartitionRecordsSorted(String propertyValue) {
|
||||||
|
return propertyValue != null ? Boolean.parseBoolean(propertyValue) : DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
@@ -1578,7 +1578,6 @@ public class HoodieWriteConfig extends HoodieConfig {
|
|||||||
HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
|
HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
|
||||||
|
|
||||||
writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
|
writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void validate() {
|
private void validate() {
|
||||||
|
|||||||
@@ -0,0 +1,44 @@
|
|||||||
|
/*
|
||||||
|
* 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.table.BulkInsertPartitioner;
|
||||||
|
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A factory to generate built-in partitioner to repartition input Rows into at least
|
||||||
|
* expected number of output spark partitions for bulk insert operation.
|
||||||
|
*/
|
||||||
|
public abstract class BulkInsertInternalPartitionerWithRowsFactory {
|
||||||
|
|
||||||
|
public static BulkInsertPartitioner<Dataset<Row>> get(BulkInsertSortMode sortMode) {
|
||||||
|
switch (sortMode) {
|
||||||
|
case NONE:
|
||||||
|
return new NonSortPartitionerWithRows();
|
||||||
|
case GLOBAL_SORT:
|
||||||
|
return new GlobalSortPartitionerWithRows();
|
||||||
|
case PARTITION_SORT:
|
||||||
|
return new PartitionSortPartitionerWithRows();
|
||||||
|
default:
|
||||||
|
throw new UnsupportedOperationException("The bulk insert sort mode \"" + sortMode.name() + "\" is not supported.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,45 @@
|
|||||||
|
/*
|
||||||
|
* 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.table.BulkInsertPartitioner;
|
||||||
|
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
import org.apache.spark.sql.functions;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A built-in partitioner that does global sorting for the input Rows across partitions after repartition for bulk insert operation, corresponding to the {@code BulkInsertSortMode.GLOBAL_SORT} mode.
|
||||||
|
*/
|
||||||
|
public class GlobalSortPartitionerWithRows implements BulkInsertPartitioner<Dataset<Row>> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Dataset<Row> repartitionRecords(Dataset<Row> rows, int outputSparkPartitions) {
|
||||||
|
// Now, sort the records and line them up nicely for loading.
|
||||||
|
// Let's use "partitionPath + key" as the sort key.
|
||||||
|
return rows.sort(functions.col(HoodieRecord.PARTITION_PATH_METADATA_FIELD), functions.col(HoodieRecord.RECORD_KEY_METADATA_FIELD))
|
||||||
|
.coalesce(outputSparkPartitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean arePartitionRecordsSorted() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,42 @@
|
|||||||
|
/*
|
||||||
|
* 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.table.BulkInsertPartitioner;
|
||||||
|
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A built-in partitioner that only does coalesce for input Rows for bulk insert operation,
|
||||||
|
* corresponding to the {@code BulkInsertSortMode.NONE} mode.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class NonSortPartitionerWithRows implements BulkInsertPartitioner<Dataset<Row>> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Dataset<Row> repartitionRecords(Dataset<Row> rows, int outputSparkPartitions) {
|
||||||
|
return rows.coalesce(outputSparkPartitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean arePartitionRecordsSorted() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,41 @@
|
|||||||
|
/*
|
||||||
|
* 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.table.BulkInsertPartitioner;
|
||||||
|
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A built-in partitioner that does local sorting for each spark partitions after coalesce for bulk insert operation, corresponding to the {@code BulkInsertSortMode.PARTITION_SORT} mode.
|
||||||
|
*/
|
||||||
|
public class PartitionSortPartitionerWithRows implements BulkInsertPartitioner<Dataset<Row>> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Dataset<Row> repartitionRecords(Dataset<Row> rows, int outputSparkPartitions) {
|
||||||
|
return rows.coalesce(outputSparkPartitions).sortWithinPartitions(HoodieRecord.PARTITION_PATH_METADATA_FIELD, HoodieRecord.RECORD_KEY_METADATA_FIELD);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean arePartitionRecordsSorted() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,139 @@
|
|||||||
|
/*
|
||||||
|
* 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.HoodieClientTestHarness;
|
||||||
|
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
||||||
|
|
||||||
|
import org.apache.spark.api.java.function.MapPartitionsFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unit tests {@link BulkInsertPartitioner}s with Rows.
|
||||||
|
*/
|
||||||
|
public class TestBulkInsertInternalPartitionerForRows extends HoodieClientTestHarness {
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
initSparkContexts("TestBulkInsertInternalPartitionerForRows");
|
||||||
|
initPath();
|
||||||
|
initFileSystem();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
cleanupResources();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Stream<Arguments> configParams() {
|
||||||
|
Object[][] data = new Object[][] {
|
||||||
|
{BulkInsertSortMode.GLOBAL_SORT, true, true},
|
||||||
|
{BulkInsertSortMode.PARTITION_SORT, false, true},
|
||||||
|
{BulkInsertSortMode.NONE, false, false}
|
||||||
|
};
|
||||||
|
return Stream.of(data).map(Arguments::of);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest(name = "[{index}] {0}")
|
||||||
|
@MethodSource("configParams")
|
||||||
|
public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode,
|
||||||
|
boolean isGloballySorted, boolean isLocallySorted)
|
||||||
|
throws Exception {
|
||||||
|
Dataset<Row> records1 = generateTestRecords();
|
||||||
|
Dataset<Row> records2 = generateTestRecords();
|
||||||
|
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerWithRowsFactory.get(sortMode),
|
||||||
|
records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1));
|
||||||
|
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerWithRowsFactory.get(sortMode),
|
||||||
|
records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner,
|
||||||
|
Dataset<Row> rows,
|
||||||
|
boolean isGloballySorted, boolean isLocallySorted,
|
||||||
|
Map<String, Long> expectedPartitionNumRecords) {
|
||||||
|
int numPartitions = 2;
|
||||||
|
Dataset<Row> actualRecords = (Dataset<Row>) partitioner.repartitionRecords(rows, numPartitions);
|
||||||
|
List<Row> collectedActualRecords = actualRecords.collectAsList();
|
||||||
|
if (isGloballySorted) {
|
||||||
|
// Verify global order
|
||||||
|
verifyRowsAscendingOrder(collectedActualRecords);
|
||||||
|
} else if (isLocallySorted) {
|
||||||
|
// Verify local order
|
||||||
|
actualRecords.mapPartitions((MapPartitionsFunction<Row, Object>) input -> {
|
||||||
|
List<Row> partitionRows = new ArrayList<>();
|
||||||
|
while (input.hasNext()) {
|
||||||
|
partitionRows.add(input.next());
|
||||||
|
}
|
||||||
|
verifyRowsAscendingOrder(partitionRows);
|
||||||
|
return Collections.emptyList().iterator();
|
||||||
|
}, SparkDatasetTestUtils.ENCODER);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Verify number of records per partition path
|
||||||
|
Map<String, Long> actualPartitionNumRecords = new HashMap<>();
|
||||||
|
for (Row record : collectedActualRecords) {
|
||||||
|
String partitionPath = record.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
|
||||||
|
actualPartitionNumRecords.put(partitionPath,
|
||||||
|
actualPartitionNumRecords.getOrDefault(partitionPath, 0L) + 1);
|
||||||
|
}
|
||||||
|
assertEquals(expectedPartitionNumRecords, actualPartitionNumRecords);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Map<String, Long> generateExpectedPartitionNumRecords(Dataset<Row> rows) {
|
||||||
|
Dataset<Row> toReturn = rows.groupBy(HoodieRecord.PARTITION_PATH_METADATA_FIELD).count();
|
||||||
|
List<Row> result = toReturn.collectAsList();
|
||||||
|
Map<String, Long> returnMap = new HashMap<>();
|
||||||
|
for (Row row : result) {
|
||||||
|
returnMap.put(row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD), (Long) row.getAs("count"));
|
||||||
|
}
|
||||||
|
return returnMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Dataset<Row> generateTestRecords() {
|
||||||
|
Dataset<Row> rowsPart1 = SparkDatasetTestUtils.getRandomRows(sqlContext, 100, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, false);
|
||||||
|
Dataset<Row> rowsPart2 = SparkDatasetTestUtils.getRandomRows(sqlContext, 150, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, false);
|
||||||
|
return rowsPart1.union(rowsPart2);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void verifyRowsAscendingOrder(List<Row> records) {
|
||||||
|
List<Row> expectedRecords = new ArrayList<>(records);
|
||||||
|
Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + o.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD))));
|
||||||
|
assertEquals(expectedRecords, records);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
@@ -49,6 +49,8 @@ import org.apache.log4j.LogManager;
|
|||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@@ -98,6 +100,25 @@ public class DataSourceUtils {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a UserDefinedBulkInsertPartitionerRows class via reflection,
|
||||||
|
* <br>
|
||||||
|
* if the class name of UserDefinedBulkInsertPartitioner is configured through the HoodieWriteConfig.
|
||||||
|
*
|
||||||
|
* @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass()
|
||||||
|
*/
|
||||||
|
public static Option<BulkInsertPartitioner<Dataset<Row>>> createUserDefinedBulkInsertPartitionerWithRows(HoodieWriteConfig config)
|
||||||
|
throws HoodieException {
|
||||||
|
String bulkInsertPartitionerClass = config.getUserDefinedBulkInsertPartitionerClass();
|
||||||
|
try {
|
||||||
|
return StringUtils.isNullOrEmpty(bulkInsertPartitionerClass)
|
||||||
|
? Option.empty() :
|
||||||
|
Option.of((BulkInsertPartitioner) ReflectionUtils.loadClass(bulkInsertPartitionerClass));
|
||||||
|
} catch (Throwable e) {
|
||||||
|
throw new HoodieException("Could not create UserDefinedBulkInsertPartitionerRows class " + bulkInsertPartitionerClass, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a payload class via reflection, passing in an ordering/precombine value.
|
* Create a payload class via reflection, passing in an ordering/precombine value.
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -31,7 +31,9 @@ import org.apache.spark.sql.types.StructType;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -48,15 +50,17 @@ public class BulkInsertDataInternalWriterHelper {
|
|||||||
private final HoodieTable hoodieTable;
|
private final HoodieTable hoodieTable;
|
||||||
private final HoodieWriteConfig writeConfig;
|
private final HoodieWriteConfig writeConfig;
|
||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
|
private final Boolean arePartitionRecordsSorted;
|
||||||
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
|
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
|
||||||
|
|
||||||
private HoodieRowCreateHandle handle;
|
private HoodieRowCreateHandle handle;
|
||||||
private String lastKnownPartitionPath = null;
|
private String lastKnownPartitionPath = null;
|
||||||
private String fileIdPrefix;
|
private String fileIdPrefix;
|
||||||
private int numFilesWritten = 0;
|
private int numFilesWritten = 0;
|
||||||
|
private Map<String, HoodieRowCreateHandle> handles = new HashMap<>();
|
||||||
|
|
||||||
public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||||
String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType) {
|
String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, boolean arePartitionRecordsSorted) {
|
||||||
this.hoodieTable = hoodieTable;
|
this.hoodieTable = hoodieTable;
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig;
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
@@ -64,6 +68,7 @@ public class BulkInsertDataInternalWriterHelper {
|
|||||||
this.taskId = taskId;
|
this.taskId = taskId;
|
||||||
this.taskEpochId = taskEpochId;
|
this.taskEpochId = taskEpochId;
|
||||||
this.structType = structType;
|
this.structType = structType;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
this.fileIdPrefix = UUID.randomUUID().toString();
|
this.fileIdPrefix = UUID.randomUUID().toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -74,7 +79,7 @@ public class BulkInsertDataInternalWriterHelper {
|
|||||||
|
|
||||||
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
|
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
|
||||||
LOG.info("Creating new file for partition path " + partitionPath);
|
LOG.info("Creating new file for partition path " + partitionPath);
|
||||||
createNewHandle(partitionPath);
|
handle = getRowCreateHandle(partitionPath);
|
||||||
lastKnownPartitionPath = partitionPath;
|
lastKnownPartitionPath = partitionPath;
|
||||||
}
|
}
|
||||||
handle.write(record);
|
handle.write(record);
|
||||||
@@ -92,19 +97,30 @@ public class BulkInsertDataInternalWriterHelper {
|
|||||||
public void abort() {
|
public void abort() {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createNewHandle(String partitionPath) throws IOException {
|
private HoodieRowCreateHandle getRowCreateHandle(String partitionPath) throws IOException {
|
||||||
if (null != handle) {
|
if (!handles.containsKey(partitionPath)) { // if there is no handle corresponding to the partition path
|
||||||
close();
|
// if records are sorted, we can close all existing handles
|
||||||
|
if (arePartitionRecordsSorted) {
|
||||||
|
close();
|
||||||
|
}
|
||||||
|
handles.put(partitionPath, new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
|
||||||
|
instantTime, taskPartitionId, taskId, taskEpochId, structType));
|
||||||
|
} else if (!handles.get(partitionPath).canWrite()) {
|
||||||
|
// even if there is a handle to the partition path, it could have reached its max size threshold. So, we close the handle here and
|
||||||
|
// create a new one.
|
||||||
|
writeStatusList.add(handles.remove(partitionPath).close());
|
||||||
|
handles.put(partitionPath, new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
|
||||||
|
instantTime, taskPartitionId, taskId, taskEpochId, structType));
|
||||||
}
|
}
|
||||||
handle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
|
return handles.get(partitionPath);
|
||||||
instantTime, taskPartitionId, taskId, taskEpochId, structType);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
if (null != handle) {
|
for (HoodieRowCreateHandle rowCreateHandle: handles.values()) {
|
||||||
writeStatusList.add(handle.close());
|
writeStatusList.add(rowCreateHandle.close());
|
||||||
handle = null;
|
|
||||||
}
|
}
|
||||||
|
handles.clear();
|
||||||
|
handle = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getNextFileId() {
|
private String getNextFileId() {
|
||||||
|
|||||||
@@ -30,7 +30,9 @@ import org.apache.spark.sql.Row;
|
|||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
@@ -61,13 +63,40 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size,
|
protected void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size,
|
||||||
Option<List<String>> fileAbsPaths, Option<List<String>> fileNames) {
|
Option<List<String>> fileAbsPaths, Option<List<String>> fileNames) {
|
||||||
assertEquals(batches, writeStatuses.size());
|
assertWriteStatuses(writeStatuses, batches, size, false, fileAbsPaths, fileNames);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size, boolean areRecordsSorted,
|
||||||
|
Option<List<String>> fileAbsPaths, Option<List<String>> fileNames) {
|
||||||
|
if (areRecordsSorted) {
|
||||||
|
assertEquals(batches, writeStatuses.size());
|
||||||
|
} else {
|
||||||
|
assertEquals(Math.min(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS.length, batches), writeStatuses.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, Long> sizeMap = new HashMap<>();
|
||||||
|
if (!areRecordsSorted) {
|
||||||
|
// <size> no of records are written per batch. Every 4th batch goes into same writeStatus. So, populating the size expected
|
||||||
|
// per write status
|
||||||
|
for (int i = 0; i < batches; i++) {
|
||||||
|
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[i % 3];
|
||||||
|
if (!sizeMap.containsKey(partitionPath)) {
|
||||||
|
sizeMap.put(partitionPath, 0L);
|
||||||
|
}
|
||||||
|
sizeMap.put(partitionPath, sizeMap.get(partitionPath) + size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
int counter = 0;
|
int counter = 0;
|
||||||
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
|
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
|
||||||
// verify write status
|
// verify write status
|
||||||
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath());
|
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath());
|
||||||
assertEquals(writeStatus.getTotalRecords(), size);
|
if (areRecordsSorted) {
|
||||||
|
assertEquals(writeStatus.getTotalRecords(), size);
|
||||||
|
} else {
|
||||||
|
assertEquals(writeStatus.getTotalRecords(), sizeMap.get(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]));
|
||||||
|
}
|
||||||
assertNull(writeStatus.getGlobalError());
|
assertNull(writeStatus.getGlobalError());
|
||||||
assertEquals(writeStatus.getFailedRowsSize(), 0);
|
assertEquals(writeStatus.getFailedRowsSize(), 0);
|
||||||
assertEquals(writeStatus.getTotalErrorRecords(), 0);
|
assertEquals(writeStatus.getTotalErrorRecords(), 0);
|
||||||
@@ -82,8 +111,13 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
|
|||||||
.substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1));
|
.substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1));
|
||||||
}
|
}
|
||||||
HoodieWriteStat writeStat = writeStatus.getStat();
|
HoodieWriteStat writeStat = writeStatus.getStat();
|
||||||
assertEquals(size, writeStat.getNumInserts());
|
if (areRecordsSorted) {
|
||||||
assertEquals(size, writeStat.getNumWrites());
|
assertEquals(size, writeStat.getNumInserts());
|
||||||
|
assertEquals(size, writeStat.getNumWrites());
|
||||||
|
} else {
|
||||||
|
assertEquals(sizeMap.get(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]), writeStat.getNumInserts());
|
||||||
|
assertEquals(sizeMap.get(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]), writeStat.getNumWrites());
|
||||||
|
}
|
||||||
assertEquals(fileId, writeStat.getFileId());
|
assertEquals(fileId, writeStat.getFileId());
|
||||||
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
|
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
|
||||||
assertEquals(0, writeStat.getNumDeletes());
|
assertEquals(0, writeStat.getNumDeletes());
|
||||||
|
|||||||
@@ -18,17 +18,12 @@
|
|||||||
|
|
||||||
package org.apache.hudi;
|
package org.apache.hudi;
|
||||||
|
|
||||||
import static org.apache.spark.sql.functions.callUDF;
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
import org.apache.hudi.common.config.TypedProperties;
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.util.ReflectionUtils;
|
import org.apache.hudi.common.util.ReflectionUtils;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||||
|
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
@@ -40,8 +35,16 @@ import org.apache.spark.sql.api.java.UDF1;
|
|||||||
import org.apache.spark.sql.functions;
|
import org.apache.spark.sql.functions;
|
||||||
import org.apache.spark.sql.types.DataTypes;
|
import org.apache.spark.sql.types.DataTypes;
|
||||||
import org.apache.spark.sql.types.StructType;
|
import org.apache.spark.sql.types.StructType;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import scala.collection.JavaConverters;
|
import scala.collection.JavaConverters;
|
||||||
|
|
||||||
|
import static org.apache.spark.sql.functions.callUDF;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
|
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
|
||||||
*/
|
*/
|
||||||
@@ -60,12 +63,13 @@ public class HoodieDatasetBulkInsertHelper {
|
|||||||
* 4. Sorts input dataset by hoodie partition path and record key
|
* 4. Sorts input dataset by hoodie partition path and record key
|
||||||
*
|
*
|
||||||
* @param sqlContext SQL Context
|
* @param sqlContext SQL Context
|
||||||
* @param config Hoodie Write Config
|
* @param config Hoodie Write Config
|
||||||
* @param rows Spark Input dataset
|
* @param rows Spark Input dataset
|
||||||
* @return hoodie dataset which is ready for bulk insert.
|
* @return hoodie dataset which is ready for bulk insert.
|
||||||
*/
|
*/
|
||||||
public static Dataset<Row> prepareHoodieDatasetForBulkInsert(SQLContext sqlContext,
|
public static Dataset<Row> prepareHoodieDatasetForBulkInsert(SQLContext sqlContext,
|
||||||
HoodieWriteConfig config, Dataset<Row> rows, String structName, String recordNamespace) {
|
HoodieWriteConfig config, Dataset<Row> rows, String structName, String recordNamespace,
|
||||||
|
BulkInsertPartitioner<Dataset<Row>> bulkInsertPartitionerRows) {
|
||||||
List<Column> originalFields =
|
List<Column> originalFields =
|
||||||
Arrays.stream(rows.schema().fields()).map(f -> new Column(f.name())).collect(Collectors.toList());
|
Arrays.stream(rows.schema().fields()).map(f -> new Column(f.name())).collect(Collectors.toList());
|
||||||
|
|
||||||
@@ -101,8 +105,6 @@ public class HoodieDatasetBulkInsertHelper {
|
|||||||
Dataset<Row> colOrderedDataset = rowDatasetWithHoodieColumns.select(
|
Dataset<Row> colOrderedDataset = rowDatasetWithHoodieColumns.select(
|
||||||
JavaConverters.collectionAsScalaIterableConverter(orderedFields).asScala().toSeq());
|
JavaConverters.collectionAsScalaIterableConverter(orderedFields).asScala().toSeq());
|
||||||
|
|
||||||
return colOrderedDataset
|
return bulkInsertPartitionerRows.repartitionRecords(colOrderedDataset, config.getBulkInsertShuffleParallelism());
|
||||||
.sort(functions.col(HoodieRecord.PARTITION_PATH_METADATA_FIELD), functions.col(HoodieRecord.RECORD_KEY_METADATA_FIELD))
|
|
||||||
.coalesce(config.getBulkInsertShuffleParallelism());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,7 +19,6 @@ package org.apache.hudi
|
|||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
import org.apache.avro.generic.GenericRecord
|
import org.apache.avro.generic.GenericRecord
|
||||||
import org.apache.hadoop.conf.Configuration
|
import org.apache.hadoop.conf.Configuration
|
||||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||||
@@ -34,13 +33,15 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
||||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
|
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
|
||||||
import org.apache.hudi.config.HoodieWriteConfig
|
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
|
||||||
import org.apache.hudi.exception.HoodieException
|
import org.apache.hudi.exception.HoodieException
|
||||||
|
import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory
|
||||||
import org.apache.hudi.hive.util.ConfigUtils
|
import org.apache.hudi.hive.util.ConfigUtils
|
||||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||||
|
import org.apache.hudi.table.BulkInsertPartitioner
|
||||||
import org.apache.log4j.LogManager
|
import org.apache.log4j.LogManager
|
||||||
import org.apache.spark.SPARK_VERSION
|
import org.apache.spark.SPARK_VERSION
|
||||||
import org.apache.spark.SparkContext
|
import org.apache.spark.SparkContext
|
||||||
@@ -50,7 +51,7 @@ import org.apache.spark.sql.hudi.HoodieSqlUtils
|
|||||||
import org.apache.spark.sql.internal.SQLConf
|
import org.apache.spark.sql.internal.SQLConf
|
||||||
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
||||||
import org.apache.spark.sql.types.StructType
|
import org.apache.spark.sql.types.StructType
|
||||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
|
import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession}
|
||||||
|
|
||||||
import scala.collection.JavaConversions._
|
import scala.collection.JavaConversions._
|
||||||
import scala.collection.mutable.ListBuffer
|
import scala.collection.mutable.ListBuffer
|
||||||
@@ -335,7 +336,17 @@ object HoodieSparkSqlWriter {
|
|||||||
}
|
}
|
||||||
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
|
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
|
||||||
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
|
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
|
||||||
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
|
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
|
||||||
|
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (userDefinedBulkInsertPartitionerOpt.isPresent) {
|
||||||
|
userDefinedBulkInsertPartitionerOpt.get
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig.getBulkInsertSortMode)
|
||||||
|
}
|
||||||
|
val arePartitionRecordsSorted = bulkInsertPartitionerRows.arePartitionRecordsSorted();
|
||||||
|
parameters.updated(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, arePartitionRecordsSorted.toString)
|
||||||
|
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace,
|
||||||
|
bulkInsertPartitionerRows)
|
||||||
if (SPARK_VERSION.startsWith("2.")) {
|
if (SPARK_VERSION.startsWith("2.")) {
|
||||||
hoodieDF.write.format("org.apache.hudi.internal")
|
hoodieDF.write.format("org.apache.hudi.internal")
|
||||||
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
|
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
|
||||||
|
|||||||
@@ -35,6 +35,8 @@ import org.apache.avro.generic.GenericData;
|
|||||||
import org.apache.avro.generic.GenericFixed;
|
import org.apache.avro.generic.GenericFixed;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.extension.ExtendWith;
|
import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
@@ -162,6 +164,25 @@ public class TestDataSourceUtils {
|
|||||||
assertThat(optionCaptor.getValue().get(), is(instanceOf(NoOpBulkInsertPartitioner.class)));
|
assertThat(optionCaptor.getValue().get(), is(instanceOf(NoOpBulkInsertPartitioner.class)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateUserDefinedBulkInsertPartitionerRowsWithInValidPartitioner() throws HoodieException {
|
||||||
|
config = HoodieWriteConfig.newBuilder().withPath("/").withUserDefinedBulkInsertPartitionerClass("NonExistantUserDefinedClass").build();
|
||||||
|
|
||||||
|
Exception exception = assertThrows(HoodieException.class, () -> {
|
||||||
|
DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(config);
|
||||||
|
});
|
||||||
|
|
||||||
|
assertThat(exception.getMessage(), containsString("Could not create UserDefinedBulkInsertPartitionerRows"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateUserDefinedBulkInsertPartitionerRowsWithValidPartitioner() throws HoodieException {
|
||||||
|
config = HoodieWriteConfig.newBuilder().withPath("/").withUserDefinedBulkInsertPartitionerClass(NoOpBulkInsertPartitionerRows.class.getName()).build();
|
||||||
|
|
||||||
|
Option<BulkInsertPartitioner<Dataset<Row>>> partitioner = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(config);
|
||||||
|
assertThat(partitioner.isPresent(), is(true));
|
||||||
|
}
|
||||||
|
|
||||||
private void setAndVerifyHoodieWriteClientWith(final String partitionerClassName) {
|
private void setAndVerifyHoodieWriteClientWith(final String partitionerClassName) {
|
||||||
config = HoodieWriteConfig.newBuilder().withPath(config.getBasePath())
|
config = HoodieWriteConfig.newBuilder().withPath(config.getBasePath())
|
||||||
.withUserDefinedBulkInsertPartitionerClass(partitionerClassName)
|
.withUserDefinedBulkInsertPartitionerClass(partitionerClassName)
|
||||||
@@ -184,4 +205,18 @@ public class TestDataSourceUtils {
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static class NoOpBulkInsertPartitionerRows
|
||||||
|
implements BulkInsertPartitioner<Dataset<Row>> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Dataset<Row> repartitionRecords(Dataset<Row> records, int outputSparkPartitions) {
|
||||||
|
return records;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean arePartitionRecordsSorted() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -20,6 +20,7 @@ package org.apache.hudi;
|
|||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.util.FileIOUtils;
|
import org.apache.hudi.common.util.FileIOUtils;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
|
||||||
import org.apache.hudi.testutils.DataSourceTestUtils;
|
import org.apache.hudi.testutils.DataSourceTestUtils;
|
||||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||||
|
|
||||||
@@ -62,7 +63,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|||||||
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
|
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
|
||||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||||
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace",
|
||||||
|
new NonSortPartitionerWithRows());
|
||||||
StructType resultSchema = result.schema();
|
StructType resultSchema = result.schema();
|
||||||
|
|
||||||
assertEquals(result.count(), 10);
|
assertEquals(result.count(), 10);
|
||||||
@@ -117,7 +119,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|||||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||||
try {
|
try {
|
||||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||||
|
"testNamespace", new NonSortPartitionerWithRows());
|
||||||
fail("Should have thrown exception");
|
fail("Should have thrown exception");
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// ignore
|
// ignore
|
||||||
@@ -127,7 +130,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|||||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||||
dataset = sqlContext.createDataFrame(rows, structType);
|
dataset = sqlContext.createDataFrame(rows, structType);
|
||||||
try {
|
try {
|
||||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||||
|
"testNamespace", new NonSortPartitionerWithRows());
|
||||||
fail("Should have thrown exception");
|
fail("Should have thrown exception");
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// ignore
|
// ignore
|
||||||
@@ -137,7 +141,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|||||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||||
dataset = sqlContext.createDataFrame(rows, structType);
|
dataset = sqlContext.createDataFrame(rows, structType);
|
||||||
try {
|
try {
|
||||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||||
|
"testNamespace", new NonSortPartitionerWithRows());
|
||||||
fail("Should have thrown exception");
|
fail("Should have thrown exception");
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// ignore
|
// ignore
|
||||||
@@ -147,7 +152,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|||||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||||
dataset = sqlContext.createDataFrame(rows, structType);
|
dataset = sqlContext.createDataFrame(rows, structType);
|
||||||
try {
|
try {
|
||||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
||||||
|
"testNamespace", new NonSortPartitionerWithRows());
|
||||||
fail("Should have thrown exception");
|
fail("Should have thrown exception");
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// ignore
|
// ignore
|
||||||
|
|||||||
@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
|||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||||
import org.apache.hudi.exception.HoodieException
|
import org.apache.hudi.exception.HoodieException
|
||||||
|
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode
|
||||||
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
||||||
import org.apache.hudi.hive.HiveSyncConfig
|
import org.apache.hudi.hive.HiveSyncConfig
|
||||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||||
@@ -119,9 +120,9 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
List(BulkInsertSortMode.GLOBAL_SORT.name(), BulkInsertSortMode.NONE.name(), BulkInsertSortMode.PARTITION_SORT.name())
|
||||||
.foreach(tableType => {
|
.foreach(sortMode => {
|
||||||
test("test bulk insert dataset with datasource impl for " + tableType) {
|
test("test_bulk_insert_for_" + sortMode) {
|
||||||
initSparkContext("test_bulk_insert_datasource")
|
initSparkContext("test_bulk_insert_datasource")
|
||||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||||
try {
|
try {
|
||||||
@@ -131,7 +132,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
|||||||
//create a new table
|
//create a new table
|
||||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> tableType,
|
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||||
@@ -143,7 +144,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
|||||||
// generate the inserts
|
// generate the inserts
|
||||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
val records = DataSourceTestUtils.generateRandomRows(1000)
|
||||||
val recordsSeq = convertRowListToSeq(records)
|
val recordsSeq = convertRowListToSeq(records)
|
||||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||||
// write to Hudi
|
// write to Hudi
|
||||||
|
|||||||
@@ -253,6 +253,11 @@
|
|||||||
<artifactId>junit-jupiter-api</artifactId>
|
<artifactId>junit-jupiter-api</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.junit.jupiter</groupId>
|
||||||
|
<artifactId>junit-jupiter-params</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|||||||
@@ -19,6 +19,7 @@
|
|||||||
package org.apache.hudi.internal;
|
package org.apache.hudi.internal;
|
||||||
|
|
||||||
import org.apache.hudi.DataSourceUtils;
|
import org.apache.hudi.DataSourceUtils;
|
||||||
|
import org.apache.hudi.config.HoodieInternalConfig;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
@@ -62,7 +63,10 @@ public class DefaultSource extends BaseDefaultSource implements DataSourceV2,
|
|||||||
String tblName = options.get(HoodieWriteConfig.TABLE_NAME.key()).get();
|
String tblName = options.get(HoodieWriteConfig.TABLE_NAME.key()).get();
|
||||||
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
|
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
|
||||||
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA.key()).get(), path, tblName, options.asMap());
|
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA.key()).get(), path, tblName, options.asMap());
|
||||||
|
boolean arePartitionRecordsSorted = HoodieInternalConfig.getBulkInsertIsPartitionRecordsSorted(
|
||||||
|
options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).isPresent()
|
||||||
|
? options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).get() : null);
|
||||||
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
|
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
|
||||||
getConfiguration()));
|
getConfiguration(), arePartitionRecordsSorted));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -37,9 +37,9 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRo
|
|||||||
|
|
||||||
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||||
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
|
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
|
||||||
StructType structType) {
|
StructType structType, boolean arePartitionRecordsSorted) {
|
||||||
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
|
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
|
||||||
writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType);
|
writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -35,18 +35,20 @@ public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFact
|
|||||||
private final HoodieTable hoodieTable;
|
private final HoodieTable hoodieTable;
|
||||||
private final HoodieWriteConfig writeConfig;
|
private final HoodieWriteConfig writeConfig;
|
||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
|
private final boolean arePartitionRecordsSorted;
|
||||||
|
|
||||||
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||||
String instantTime, StructType structType) {
|
String instantTime, StructType structType, boolean arePartitionRecordsSorted) {
|
||||||
this.hoodieTable = hoodieTable;
|
this.hoodieTable = hoodieTable;
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig;
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
this.structType = structType;
|
this.structType = structType;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
|
public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
|
||||||
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId,
|
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId,
|
||||||
structType);
|
structType, arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -45,12 +45,14 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
|
|||||||
private final HoodieWriteConfig writeConfig;
|
private final HoodieWriteConfig writeConfig;
|
||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
|
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
|
||||||
|
private final Boolean arePartitionRecordsSorted;
|
||||||
|
|
||||||
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
||||||
SparkSession sparkSession, Configuration configuration) {
|
SparkSession sparkSession, Configuration configuration, boolean arePartitionRecordsSorted) {
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig;
|
||||||
this.structType = structType;
|
this.structType = structType;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
|
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
|
||||||
sparkSession, configuration);
|
sparkSession, configuration);
|
||||||
}
|
}
|
||||||
@@ -60,7 +62,7 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
|
|||||||
dataSourceInternalWriterHelper.createInflightCommit();
|
dataSourceInternalWriterHelper.createInflightCommit();
|
||||||
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
|
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
|
||||||
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
|
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
|
||||||
writeConfig, instantTime, structType);
|
writeConfig, instantTime, structType, arePartitionRecordsSorted);
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
|
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -28,9 +28,13 @@ import org.apache.spark.sql.Dataset;
|
|||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
import org.apache.spark.sql.catalyst.InternalRow;
|
import org.apache.spark.sql.catalyst.InternalRow;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
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.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
||||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
||||||
@@ -46,16 +50,26 @@ import static org.junit.jupiter.api.Assertions.fail;
|
|||||||
public class TestHoodieBulkInsertDataInternalWriter extends
|
public class TestHoodieBulkInsertDataInternalWriter extends
|
||||||
HoodieBulkInsertInternalWriterTestBase {
|
HoodieBulkInsertInternalWriterTestBase {
|
||||||
|
|
||||||
@Test
|
private static Stream<Arguments> configParams() {
|
||||||
public void testDataInternalWriter() throws Exception {
|
Object[][] data = new Object[][] {
|
||||||
|
{true},
|
||||||
|
{false}
|
||||||
|
};
|
||||||
|
return Stream.of(data).map(Arguments::of);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("configParams")
|
||||||
|
public void testDataInternalWriter(boolean sorted) throws Exception {
|
||||||
// init config and table
|
// init config and table
|
||||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||||
// execute N rounds
|
// execute N rounds
|
||||||
for (int i = 0; i < 5; i++) {
|
for (int i = 0; i < 3; i++) {
|
||||||
String instantTime = "00" + i;
|
String instantTime = "00" + i;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
|
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE,
|
||||||
|
sorted);
|
||||||
|
|
||||||
int size = 10 + RANDOM.nextInt(1000);
|
int size = 10 + RANDOM.nextInt(1000);
|
||||||
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
|
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
|
||||||
@@ -78,7 +92,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
|||||||
Option<List<String>> fileNames = Option.of(new ArrayList<>());
|
Option<List<String>> fileNames = Option.of(new ArrayList<>());
|
||||||
|
|
||||||
// verify write statuses
|
// verify write statuses
|
||||||
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames);
|
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, sorted, fileAbsPaths, fileNames);
|
||||||
|
|
||||||
// verify rows
|
// verify rows
|
||||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
||||||
@@ -99,7 +113,8 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
|||||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||||
|
|
||||||
String instantTime = "001";
|
String instantTime = "001";
|
||||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
|
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE,
|
||||||
|
false);
|
||||||
|
|
||||||
int size = 10 + RANDOM.nextInt(100);
|
int size = 10 + RANDOM.nextInt(100);
|
||||||
int totalFailures = 5;
|
int totalFailures = 5;
|
||||||
@@ -131,7 +146,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
|||||||
Option<List<String>> fileAbsPaths = Option.of(new ArrayList<>());
|
Option<List<String>> fileAbsPaths = Option.of(new ArrayList<>());
|
||||||
Option<List<String>> fileNames = Option.of(new ArrayList<>());
|
Option<List<String>> fileNames = Option.of(new ArrayList<>());
|
||||||
// verify write statuses
|
// verify write statuses
|
||||||
assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames);
|
assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, false, fileAbsPaths, fileNames);
|
||||||
|
|
||||||
// verify rows
|
// verify rows
|
||||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
||||||
|
|||||||
@@ -52,7 +52,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
|||||||
String instantTime = "001";
|
String instantTime = "001";
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
|
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
|
||||||
|
|
||||||
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
|
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
|
||||||
@@ -98,7 +98,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
|||||||
String instantTime = "00" + i;
|
String instantTime = "00" + i;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
|
|
||||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||||
Dataset<Row> totalInputRows = null;
|
Dataset<Row> totalInputRows = null;
|
||||||
@@ -142,7 +142,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
|||||||
String instantTime = "00" + i;
|
String instantTime = "00" + i;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||||
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
|
|
||||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||||
Dataset<Row> totalInputRows = null;
|
Dataset<Row> totalInputRows = null;
|
||||||
@@ -189,7 +189,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
|||||||
String instantTime0 = "00" + 0;
|
String instantTime0 = "00" + 0;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
HoodieDataSourceInternalWriter dataSourceInternalWriter =
|
||||||
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
|
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
|
||||||
|
|
||||||
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
|
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
|
||||||
@@ -227,7 +227,7 @@ public class TestHoodieDataSourceInternalWriter extends
|
|||||||
// 2nd batch. abort in the end
|
// 2nd batch. abort in the end
|
||||||
String instantTime1 = "00" + 1;
|
String instantTime1 = "00" + 1;
|
||||||
dataSourceInternalWriter =
|
dataSourceInternalWriter =
|
||||||
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
|
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
|
||||||
|
|
||||||
for (int j = 0; j < batches; j++) {
|
for (int j = 0; j < batches; j++) {
|
||||||
|
|||||||
@@ -226,6 +226,11 @@
|
|||||||
<artifactId>junit-jupiter-api</artifactId>
|
<artifactId>junit-jupiter-api</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.junit.jupiter</groupId>
|
||||||
|
<artifactId>junit-jupiter-params</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|||||||
@@ -19,6 +19,7 @@
|
|||||||
package org.apache.hudi.spark3.internal;
|
package org.apache.hudi.spark3.internal;
|
||||||
|
|
||||||
import org.apache.hudi.DataSourceUtils;
|
import org.apache.hudi.DataSourceUtils;
|
||||||
|
import org.apache.hudi.config.HoodieInternalConfig;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.internal.BaseDefaultSource;
|
import org.apache.hudi.internal.BaseDefaultSource;
|
||||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper;
|
import org.apache.hudi.internal.DataSourceInternalWriterHelper;
|
||||||
@@ -47,9 +48,11 @@ public class DefaultSource extends BaseDefaultSource implements TableProvider {
|
|||||||
String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY);
|
String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY);
|
||||||
String path = properties.get("path");
|
String path = properties.get("path");
|
||||||
String tblName = properties.get(HoodieWriteConfig.TABLE_NAME.key());
|
String tblName = properties.get(HoodieWriteConfig.TABLE_NAME.key());
|
||||||
|
boolean arePartitionRecordsSorted = Boolean.parseBoolean(properties.getOrDefault(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED,
|
||||||
|
Boolean.toString(HoodieInternalConfig.DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED)));
|
||||||
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
|
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
|
||||||
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA.key()), path, tblName, properties);
|
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA.key()), path, tblName, properties);
|
||||||
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
|
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
|
||||||
getConfiguration());
|
getConfiguration(), arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -37,9 +37,9 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRo
|
|||||||
private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper;
|
private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper;
|
||||||
|
|
||||||
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||||
String instantTime, int taskPartitionId, long taskId, StructType structType) {
|
String instantTime, int taskPartitionId, long taskId, StructType structType, boolean arePartitionRecordsSorted) {
|
||||||
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
|
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
|
||||||
writeConfig, instantTime, taskPartitionId, taskId, 0, structType);
|
writeConfig, instantTime, taskPartitionId, taskId, 0, structType, arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -35,18 +35,20 @@ public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFact
|
|||||||
private final HoodieTable hoodieTable;
|
private final HoodieTable hoodieTable;
|
||||||
private final HoodieWriteConfig writeConfig;
|
private final HoodieWriteConfig writeConfig;
|
||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
|
private final boolean arePartitionRecordsSorted;
|
||||||
|
|
||||||
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
|
||||||
String instantTime, StructType structType) {
|
String instantTime, StructType structType, boolean arePartitionRecordsSorted) {
|
||||||
this.hoodieTable = hoodieTable;
|
this.hoodieTable = hoodieTable;
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig;
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
this.structType = structType;
|
this.structType = structType;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
|
public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
|
||||||
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId,
|
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId,
|
||||||
structType);
|
structType, arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -45,13 +45,15 @@ public class HoodieDataSourceInternalBatchWrite implements BatchWrite {
|
|||||||
private final String instantTime;
|
private final String instantTime;
|
||||||
private final HoodieWriteConfig writeConfig;
|
private final HoodieWriteConfig writeConfig;
|
||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
|
private final boolean arePartitionRecordsSorted;
|
||||||
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
|
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
|
||||||
|
|
||||||
public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
||||||
SparkSession jss, Configuration hadoopConfiguration) {
|
SparkSession jss, Configuration hadoopConfiguration, boolean arePartitionRecordsSorted) {
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig;
|
||||||
this.structType = structType;
|
this.structType = structType;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
|
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
|
||||||
jss, hadoopConfiguration);
|
jss, hadoopConfiguration);
|
||||||
}
|
}
|
||||||
@@ -61,7 +63,7 @@ public class HoodieDataSourceInternalBatchWrite implements BatchWrite {
|
|||||||
dataSourceInternalWriterHelper.createInflightCommit();
|
dataSourceInternalWriterHelper.createInflightCommit();
|
||||||
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
|
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
|
||||||
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
|
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
|
||||||
writeConfig, instantTime, structType);
|
writeConfig, instantTime, structType, arePartitionRecordsSorted);
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
|
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -37,19 +37,21 @@ public class HoodieDataSourceInternalBatchWriteBuilder implements WriteBuilder {
|
|||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
private final SparkSession jss;
|
private final SparkSession jss;
|
||||||
private final Configuration hadoopConfiguration;
|
private final Configuration hadoopConfiguration;
|
||||||
|
private final boolean arePartitionRecordsSorted;
|
||||||
|
|
||||||
public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
|
||||||
SparkSession jss, Configuration hadoopConfiguration) {
|
SparkSession jss, Configuration hadoopConfiguration, boolean arePartitionRecordsSorted) {
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
this.writeConfig = writeConfig;
|
this.writeConfig = writeConfig;
|
||||||
this.structType = structType;
|
this.structType = structType;
|
||||||
this.jss = jss;
|
this.jss = jss;
|
||||||
this.hadoopConfiguration = hadoopConfiguration;
|
this.hadoopConfiguration = hadoopConfiguration;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public BatchWrite buildForBatch() {
|
public BatchWrite buildForBatch() {
|
||||||
return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss,
|
return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss,
|
||||||
hadoopConfiguration);
|
hadoopConfiguration, arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -41,14 +41,16 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
|
|||||||
private final StructType structType;
|
private final StructType structType;
|
||||||
private final SparkSession jss;
|
private final SparkSession jss;
|
||||||
private final Configuration hadoopConfiguration;
|
private final Configuration hadoopConfiguration;
|
||||||
|
private final boolean arePartitionRecordsSorted;
|
||||||
|
|
||||||
public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config,
|
public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config,
|
||||||
StructType schema, SparkSession jss, Configuration hadoopConfiguration) {
|
StructType schema, SparkSession jss, Configuration hadoopConfiguration, boolean arePartitionRecordsSorted) {
|
||||||
this.instantTime = instantTime;
|
this.instantTime = instantTime;
|
||||||
this.writeConfig = config;
|
this.writeConfig = config;
|
||||||
this.structType = schema;
|
this.structType = schema;
|
||||||
this.jss = jss;
|
this.jss = jss;
|
||||||
this.hadoopConfiguration = hadoopConfiguration;
|
this.hadoopConfiguration = hadoopConfiguration;
|
||||||
|
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -73,6 +75,6 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
|
|||||||
@Override
|
@Override
|
||||||
public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) {
|
public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) {
|
||||||
return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss,
|
return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss,
|
||||||
hadoopConfiguration);
|
hadoopConfiguration, arePartitionRecordsSorted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -29,9 +29,13 @@ import org.apache.spark.sql.Dataset;
|
|||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
import org.apache.spark.sql.catalyst.InternalRow;
|
import org.apache.spark.sql.catalyst.InternalRow;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
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.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
||||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
||||||
@@ -47,8 +51,17 @@ import static org.junit.jupiter.api.Assertions.fail;
|
|||||||
public class TestHoodieBulkInsertDataInternalWriter extends
|
public class TestHoodieBulkInsertDataInternalWriter extends
|
||||||
HoodieBulkInsertInternalWriterTestBase {
|
HoodieBulkInsertInternalWriterTestBase {
|
||||||
|
|
||||||
@Test
|
private static Stream<Arguments> configParams() {
|
||||||
public void testDataInternalWriter() throws Exception {
|
Object[][] data = new Object[][] {
|
||||||
|
{true},
|
||||||
|
{false}
|
||||||
|
};
|
||||||
|
return Stream.of(data).map(Arguments::of);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("configParams")
|
||||||
|
public void testDataInternalWriter(boolean sorted) throws Exception {
|
||||||
// init config and table
|
// init config and table
|
||||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||||
@@ -56,7 +69,8 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
|||||||
for (int i = 0; i < 5; i++) {
|
for (int i = 0; i < 5; i++) {
|
||||||
String instantTime = "00" + i;
|
String instantTime = "00" + i;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE);
|
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE,
|
||||||
|
sorted);
|
||||||
|
|
||||||
int size = 10 + RANDOM.nextInt(1000);
|
int size = 10 + RANDOM.nextInt(1000);
|
||||||
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
|
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
|
||||||
@@ -79,7 +93,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
|||||||
Option<List<String>> fileNames = Option.of(new ArrayList<>());
|
Option<List<String>> fileNames = Option.of(new ArrayList<>());
|
||||||
|
|
||||||
// verify write statuses
|
// verify write statuses
|
||||||
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames);
|
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, sorted, fileAbsPaths, fileNames);
|
||||||
|
|
||||||
// verify rows
|
// verify rows
|
||||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
||||||
@@ -100,7 +114,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
|||||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||||
|
|
||||||
String instantTime = "001";
|
String instantTime = "001";
|
||||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE);
|
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE, false);
|
||||||
|
|
||||||
int size = 10 + RANDOM.nextInt(100);
|
int size = 10 + RANDOM.nextInt(100);
|
||||||
int totalFailures = 5;
|
int totalFailures = 5;
|
||||||
|
|||||||
@@ -56,7 +56,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
|||||||
String instantTime = "001";
|
String instantTime = "001";
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
|
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
|
||||||
|
|
||||||
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
|
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
|
||||||
@@ -103,7 +103,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
|||||||
String instantTime = "00" + i;
|
String instantTime = "00" + i;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
|
|
||||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||||
Dataset<Row> totalInputRows = null;
|
Dataset<Row> totalInputRows = null;
|
||||||
@@ -148,7 +148,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
|||||||
String instantTime = "00" + i;
|
String instantTime = "00" + i;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
|
|
||||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||||
Dataset<Row> totalInputRows = null;
|
Dataset<Row> totalInputRows = null;
|
||||||
@@ -195,7 +195,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
|||||||
String instantTime0 = "00" + 0;
|
String instantTime0 = "00" + 0;
|
||||||
// init writer
|
// init writer
|
||||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||||
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
|
|
||||||
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
|
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
|
||||||
|
|
||||||
@@ -234,7 +234,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
|||||||
// 2nd batch. abort in the end
|
// 2nd batch. abort in the end
|
||||||
String instantTime1 = "00" + 1;
|
String instantTime1 = "00" + 1;
|
||||||
dataSourceInternalBatchWrite =
|
dataSourceInternalBatchWrite =
|
||||||
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
|
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, false);
|
||||||
writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong());
|
writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong());
|
||||||
|
|
||||||
for (int j = 0; j < batches; j++) {
|
for (int j = 0; j < batches; j++) {
|
||||||
|
|||||||
Reference in New Issue
Block a user