-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4363] Support Clustering row writer to improve performance #6046
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
6bb8dd6
d94e3ec
25f1afd
fa7c100
1c91345
f9bed48
687ec04
b8e848d
5a16d35
60ef514
1355882
7300c9e
67924e5
090434e
988e487
f2bb9e6
1587f47
20f64af
9632919
2ff0b70
e75f6d0
46004b0
af01ca4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| package org.apache.hudi.client.clustering.run.strategy; | ||
|
|
||
| import org.apache.hudi.HoodieDatasetBulkInsertHelper; | ||
| import org.apache.hudi.client.WriteStatus; | ||
| import org.apache.hudi.common.data.HoodieData; | ||
| import org.apache.hudi.common.engine.HoodieEngineContext; | ||
|
|
@@ -35,6 +36,8 @@ | |
| import org.apache.avro.Schema; | ||
| import org.apache.log4j.LogManager; | ||
| import org.apache.log4j.Logger; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Row; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
@@ -54,15 +57,40 @@ public SparkSingleFileSortExecutionStrategy(HoodieTable table, | |
| super(table, engineContext, writeConfig); | ||
| } | ||
|
|
||
| @Override | ||
| public HoodieData<WriteStatus> performClusteringWithRecordsAsRow(Dataset<Row> inputRecords, | ||
| int numOutputGroups, | ||
| String instantTime, | ||
| Map<String, String> strategyParams, | ||
| Schema schema, | ||
| List<HoodieFileGroupId> fileGroupIdList, | ||
| boolean shouldPreserveHoodieMetadata, | ||
| Map<String, String> extraMetadata) { | ||
| if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { | ||
| throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); | ||
| } | ||
| LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); | ||
|
|
||
| HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() | ||
| .withBulkInsertParallelism(numOutputGroups) | ||
| .withProps(getWriteConfig().getProps()).build(); | ||
|
|
||
| // Since clustering will write to single file group using HoodieUnboundedCreateHandle, set max file size to a large value. | ||
| newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's duplicate the comment from the original method as well |
||
|
|
||
| return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, | ||
| getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata); | ||
| } | ||
|
|
||
| @Override | ||
| public HoodieData<WriteStatus> performClusteringWithRecordsRDD(HoodieData<HoodieRecord<T>> inputRecords, | ||
| int numOutputGroups, | ||
| String instantTime, | ||
| Map<String, String> strategyParams, | ||
| Schema schema, | ||
| List<HoodieFileGroupId> fileGroupIdList, | ||
| boolean preserveHoodieMetadata, | ||
| Map<String, String> extraMetadata) { | ||
| int numOutputGroups, | ||
| String instantTime, | ||
| Map<String, String> strategyParams, | ||
| Schema schema, | ||
| List<HoodieFileGroupId> fileGroupIdList, | ||
| boolean shouldPreserveHoodieMetadata, | ||
| Map<String, String> extraMetadata) { | ||
| if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { | ||
| throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); | ||
| } | ||
|
|
@@ -75,6 +103,6 @@ public HoodieData<WriteStatus> performClusteringWithRecordsRDD(HoodieData<Hoodie | |
| newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE)); | ||
|
|
||
| return (HoodieData<WriteStatus>) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, | ||
| false, getPartitioner(strategyParams, schema), true, numOutputGroups, new SingleFileHandleCreateFactory(fileGroupIdList.get(0).getFileId(), preserveHoodieMetadata)); | ||
| false, getRDDPartitioner(strategyParams, schema), true, numOutputGroups, new SingleFileHandleCreateFactory(fileGroupIdList.get(0).getFileId(), shouldPreserveHoodieMetadata)); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,41 +29,31 @@ | |
| import org.apache.hudi.common.model.RewriteAvroPayload; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.hudi.config.HoodieClusteringConfig; | ||
| import org.apache.hudi.sort.SpaceCurveSortingHelper; | ||
| import org.apache.hudi.table.BulkInsertPartitioner; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.spark.api.java.JavaRDD; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Row; | ||
|
|
||
| import java.util.Arrays; | ||
| import java.util.List; | ||
|
|
||
| /** | ||
| * A partitioner that does spatial curve optimization sorting based on specified column values for each RDD partition. | ||
| * support z-curve optimization, hilbert will come soon. | ||
| * @param <T> HoodieRecordPayload type | ||
| */ | ||
| public class RDDSpatialCurveSortPartitioner<T extends HoodieRecordPayload> | ||
| implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> { | ||
| extends SpatialCurveSortPartitionerBase<JavaRDD<HoodieRecord<T>>> { | ||
|
|
||
| private final transient HoodieSparkEngineContext sparkEngineContext; | ||
| private final String[] orderByColumns; | ||
| private final SerializableSchema schema; | ||
| private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy; | ||
| private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType; | ||
|
|
||
| public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext, | ||
| String[] orderByColumns, | ||
| HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, | ||
| HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType, | ||
| Schema schema) { | ||
| super(orderByColumns, layoutOptStrategy, curveCompositionStrategyType); | ||
| this.sparkEngineContext = sparkEngineContext; | ||
| this.orderByColumns = orderByColumns; | ||
| this.layoutOptStrategy = layoutOptStrategy; | ||
| this.curveCompositionStrategyType = curveCompositionStrategyType; | ||
| this.schema = new SerializableSchema(schema); | ||
| } | ||
|
|
||
|
|
@@ -91,27 +81,4 @@ public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> reco | |
| return hoodieRecord; | ||
| }); | ||
| } | ||
|
|
||
| private Dataset<Row> reorder(Dataset<Row> dataset, int numOutputGroups) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for cleaning that up! |
||
| if (orderByColumns.length == 0) { | ||
| // No-op | ||
| return dataset; | ||
| } | ||
|
|
||
| List<String> orderedCols = Arrays.asList(orderByColumns); | ||
|
|
||
| switch (curveCompositionStrategyType) { | ||
| case DIRECT: | ||
| return SpaceCurveSortingHelper.orderDataFrameByMappingValues(dataset, layoutOptStrategy, orderedCols, numOutputGroups); | ||
| case SAMPLE: | ||
| return SpaceCurveSortingHelper.orderDataFrameBySamplingValues(dataset, layoutOptStrategy, orderedCols, numOutputGroups); | ||
| default: | ||
| throw new UnsupportedOperationException(String.format("Unsupported space-curve curve building strategy (%s)", curveCompositionStrategyType)); | ||
| } | ||
| } | ||
|
|
||
| @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.config.HoodieClusteringConfig; | ||
| import org.apache.hudi.config.HoodieWriteConfig; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Row; | ||
|
|
||
| public class RowSpatialCurveSortPartitioner extends SpatialCurveSortPartitionerBase<Dataset<Row>> { | ||
|
|
||
| public RowSpatialCurveSortPartitioner(HoodieWriteConfig config) { | ||
| super(config.getClusteringSortColumns(), config.getLayoutOptimizationStrategy(), config.getLayoutOptimizationCurveBuildMethod()); | ||
| } | ||
|
|
||
| public RowSpatialCurveSortPartitioner(String[] orderByColumns, | ||
| HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, | ||
| HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType) { | ||
| super(orderByColumns, layoutOptStrategy, curveCompositionStrategyType); | ||
| } | ||
|
|
||
| @Override | ||
| public Dataset<Row> repartitionRecords(Dataset<Row> records, int outputPartitions) { | ||
| return reorder(records, outputPartitions); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We need to do
getBooleanOrDefault, otherwise it might NPE (due to unboxing)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
EMBEDDED_TIMELINE_SERVER_REUSE_ENABLEDis a ConfigProperty, getBoolean will handle this.