1
0

[HUDI-4040] Bulk insert Support CustomColumnsSortPartitioner with Row (#5502)

* Along the lines of RDDCustomColumnsSortPartitioner but for Row
This commit is contained in:
RexAn
2022-05-26 13:09:04 +08:00
committed by GitHub
parent 4e42ed5eae
commit 98c5c6c654
3 changed files with 112 additions and 8 deletions

View File

@@ -29,6 +29,8 @@ import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaRDD;
import java.util.Arrays;
/**
* A partitioner that does sorting based on specified column values for each RDD partition.
*
@@ -78,6 +80,7 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
}
private String[] getSortColumnName(HoodieWriteConfig config) {
return config.getUserDefinedBulkInsertPartitionerSortColumns().split(",");
return Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(","))
.map(String::trim).toArray(String[]::new);
}
}

View File

@@ -0,0 +1,60 @@
/*
* 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.config.HoodieWriteConfig;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import java.util.Arrays;
/**
* A partitioner that does sorting based on specified column values for each spark partitions.
*/
public class RowCustomColumnsSortPartitioner implements BulkInsertPartitioner<Dataset<Row>> {
private final String[] sortColumnNames;
public RowCustomColumnsSortPartitioner(HoodieWriteConfig config) {
this.sortColumnNames = getSortColumnName(config);
}
public RowCustomColumnsSortPartitioner(String[] columnNames) {
this.sortColumnNames = columnNames;
}
@Override
public Dataset<Row> repartitionRecords(Dataset<Row> records, int outputSparkPartitions) {
final String[] sortColumns = this.sortColumnNames;
return records.coalesce(outputSparkPartitions)
.sortWithinPartitions(HoodieRecord.PARTITION_PATH_METADATA_FIELD, sortColumns);
}
@Override
public boolean arePartitionRecordsSorted() {
return true;
}
private String[] getSortColumnName(HoodieWriteConfig config) {
return Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(","))
.map(String::trim).toArray(String[]::new);
}
}