Skip to content
Closed
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.execution.bulkinsert;

import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.BulkInsertPartitioner;

/**
* 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 get(BulkInsertSortMode sortMode) {
switch (sortMode) {
case NONE:
return new NonSortPartitionerWithRows();
case GLOBAL_SORT:
return new GlobalSortPartitionerWithRows();
case PARTITION_SORT:
return new RDDPartitionSortPartitionerWithRows();
default:
throw new HoodieException("The bulk insert sort mode \"" + sortMode.name() + "\" is not supported.");
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.log4j.LogManager;
import org.apache.log4j.Logger;
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>> {

private static final Logger LOG = LogManager.getLogger(GlobalSortPartitionerWithRows.class);

@Override
public Dataset<Row> repartitionRecords(Dataset<Row> rows, int outputSparkPartitions) {
LOG.warn("TEST_LOG. GlobalSortPartitionerWithRows");
// 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))
Copy link
Contributor

@zhedoubushishi zhedoubushishi Dec 15, 2020

Choose a reason for hiding this comment

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

Looks like coalesce action is not based on the sorting result.

Here is an example, if I set outputSparkPartitions to 2, the partition column is event_type:

val df = Seq(
  (100, "event_name_16", "2015-01-01T13:51:39.340396Z", "type1"),
  (101, "event_name_546", "2015-01-01T12:14:58.597216Z", "type2"),
  (104, "event_name_123", "2015-01-01T12:15:00.512679Z", "type1"),
  (108, "event_name_18", "2015-01-01T11:51:33.340396Z", "type1"),
  (109, "event_name_19", "2014-01-01T11:51:33.340396Z", "type3"),
  (110, "event_name_20", "2014-02-01T11:51:33.340396Z", "type3"),
  (105, "event_name_678", "2015-01-01T13:51:42.248818Z", "type2")
  ).toDF("event_id", "event_name", "event_ts", "event_type")

(Here I added a new column partitionID for better understanding) Based on the current logic, after sorting and coalesce, the df would become:

val df2 = df.sort(functions.col("event_type"), functions.col("event_id")).coalesce(2)
df2.withColumn("partitionID", spark_partition_id).show(false)

+--------+--------------+---------------------------+----------+-----------+
|event_id|event_name    |event_ts                   |event_type|partitionID|
+--------+--------------+---------------------------+----------+-----------+
|100     |event_name_16 |2015-01-01T13:51:39.340396Z|type1     |0          |
|108     |event_name_18 |2015-01-01T11:51:33.340396Z|type1     |0          |
|105     |event_name_678|2015-01-01T13:51:42.248818Z|type2     |0          |
|110     |event_name_20 |2014-02-01T11:51:33.340396Z|type3     |0          |
|104     |event_name_123|2015-01-01T12:15:00.512679Z|type1     |1          |
|101     |event_name_546|2015-01-01T12:14:58.597216Z|type2     |1          |
|109     |event_name_19 |2014-01-01T11:51:33.340396Z|type3     |1          |
+--------+--------------+---------------------------+----------+-----------+

You can see the coalescing result actually does not depend on the sorting result. Each spark partition id contains 3 types of Hudi partitions.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I see, thanks for bringing it up. wanted to avoid the shuffle and hence thought will rely on coalesce. let me see if there is something we could do.

.coalesce(outputSparkPartitions);
}

@Override
public boolean arePartitionRecordsSorted() {
return true;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.log4j.LogManager;
import org.apache.log4j.Logger;
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>> {

private static final Logger LOG = LogManager.getLogger(NonSortPartitionerWithRows.class);

@Override
public Dataset<Row> repartitionRecords(Dataset<Row> rows, int outputSparkPartitions) {
LOG.warn("TEST_LOG. NonSortPartitionerWithRows");
return rows.coalesce(outputSparkPartitions);
}

@Override
public boolean arePartitionRecordsSorted() {
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* 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.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.function.MapPartitionsFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
import org.apache.spark.sql.catalyst.expressions.Attribute;
import org.apache.spark.sql.types.StructType;

import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;

import scala.collection.JavaConversions;
import scala.collection.JavaConverters;

/**
* 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 RDDPartitionSortPartitionerWithRows implements BulkInsertPartitioner<Dataset<Row>> {

private static final Logger LOG = LogManager.getLogger(RDDPartitionSortPartitionerWithRows.class);

@Override
public Dataset<Row> repartitionRecords(Dataset<Row> rows, int outputSparkPartitions) {
LOG.warn("TEST_LOG. RDDPartitionSortPartitionerWithRows");
ExpressionEncoder encoder = getEncoder(rows.schema());
return rows.coalesce(outputSparkPartitions).mapPartitions((MapPartitionsFunction<Row, Row>) input -> {
// Sort locally in partition
List<Row> recordList = new ArrayList<>();
for (; input.hasNext(); ) {
recordList.add(input.next());
}
Collections.sort(recordList, Comparator.comparing(o -> (o.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + o.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD))));
return recordList.iterator();
}, encoder);
}

@Override
public boolean arePartitionRecordsSorted() {
return true;
}

private ExpressionEncoder getEncoder(StructType schema) {
List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
.map(Attribute::toAttribute).collect(Collectors.toList());
return RowEncoder.apply(schema)
.resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(),
SimpleAnalyzer$.MODULE$);
}
}
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);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
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.util.ArrayList;
Expand Down Expand Up @@ -98,6 +100,25 @@ private static Option<BulkInsertPartitioner> createUserDefinedBulkInsertPartitio
}
}

/**
* 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.
*/
Expand Down
Loading