Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -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;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -1578,7 +1578,6 @@ protected void setDefaults() {
HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());

writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION));

}

private void validate() {
Expand Down
Original file line number Diff line number Diff line change
@@ -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 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

drop Internal from the name?

Copy link
Contributor Author

@nsivabalan nsivabalan Jul 6, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Existing factory class for write client path is called BulkInsertInternalPartitionerFactory. hence named it this way. Reason is that, we have an interface called BulkInsertPartitioner. we have few out of the box partitioners and we could have user defined as well. hence the naming for these factories as internal. I can fix the name for both the factories if you prefer.


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.");
}
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
}
Original file line number Diff line number Diff line change
@@ -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);
}

}
Loading