stream2000 commented on code in PR #9199:
URL: https://github.com/apache/hudi/pull/9199#discussion_r1283899831
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BucketBulkInsertDataInternalWriterHelper.java:
##########
@@ -65,7 +71,6 @@ public void write(InternalRow row) throws IOException {
int bucketId = BucketIdentifier.getBucketId(String.valueOf(recordKey),
indexKeyFields, bucketNum);
Pair<UTF8String, Integer> fileId = Pair.of(partitionPath, bucketId);
if (lastFileId == null || !lastFileId.equals(fileId)) {
- LOG.info("Creating new file for partition path " + partitionPath);
Review Comment:
We will not always create a new file handle if we have already created one
for this partition so I delete this line. I will add it back to
`getBucketRowCreateHandle` when we do need to create a new handle.
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java:
##########
@@ -79,15 +94,19 @@ public HoodieData<WriteStatus>
performClusteringWithRecordsRDD(HoodieData<Hoodie
HoodieWriteConfig newConfig =
HoodieWriteConfig.newBuilder().withProps(props).build();
RDDConsistentBucketBulkInsertPartitioner<T> partitioner = new
RDDConsistentBucketBulkInsertPartitioner<>(getHoodieTable(), strategyParams,
preserveHoodieMetadata);
+ addHashingChildNodes(partitioner, extraMetadata);
+
+ return (HoodieData<WriteStatus>) SparkBulkInsertHelper.newInstance()
+ .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
false, partitioner, true, numOutputGroups);
+ }
+
+ private void addHashingChildNodes(ConsistentHashingBucketInsertPartitioner
partitioner, Map<String, String> extraMetadata) {
try {
List<ConsistentHashingNode> nodes =
ConsistentHashingNode.fromJsonString(extraMetadata.get(BaseConsistentHashingBucketClusteringPlanStrategy.METADATA_CHILD_NODE_KEY));
Review Comment:
Will check the nodes is non-emtpy in `addHashingChildrenNodes`
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/ConsistentBucketIndexBulkInsertPartitionerWithRows.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.ConsistentHashingNode;
+import org.apache.hudi.common.model.HoodieConsistentHashingMetadata;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.index.bucket.ConsistentBucketIdentifier;
+import org.apache.hudi.index.bucket.ConsistentBucketIndexUtils;
+import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex;
+import org.apache.hudi.keygen.BuiltinKeyGenerator;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.ConsistentHashingBucketInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+/**
+ * Bulk_insert partitioner of Spark row using consistent hashing bucket index.
+ */
+public class ConsistentBucketIndexBulkInsertPartitionerWithRows
+ implements BulkInsertPartitioner<Dataset<Row>>,
ConsistentHashingBucketInsertPartitioner {
+
+ private final HoodieTable table;
+
+ private final String indexKeyFields;
+
+ private final List<String> fileIdPfxList = new ArrayList<>();
+ private final Map<String, List<ConsistentHashingNode>> hashingChildrenNodes;
+
+ private Map<String, ConsistentBucketIdentifier> partitionToIdentifier;
+
+ private final Option<BuiltinKeyGenerator> keyGeneratorOpt;
+
+ private Map<String, Map<String, Integer>> partitionToFileIdPfxIdxMap;
+
+ private final RowRecordKeyExtractor extractor;
+
+ public ConsistentBucketIndexBulkInsertPartitionerWithRows(HoodieTable table,
boolean populateMetaFields) {
+ this.indexKeyFields = table.getConfig().getBucketIndexHashField();
+ this.table = table;
+ this.hashingChildrenNodes = new HashMap<>();
+ if (!populateMetaFields) {
+ this.keyGeneratorOpt =
HoodieSparkKeyGeneratorFactory.getKeyGenerator(table.getConfig().getProps());
+ } else {
+ this.keyGeneratorOpt = Option.empty();
+ }
+ this.extractor =
RowRecordKeyExtractor.getRowRecordKeyExtractor(populateMetaFields,
keyGeneratorOpt);
+
ValidationUtils.checkArgument(table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ),
+ "Consistent hash bucket index doesn't support CoW table");
+ }
+
+ private ConsistentBucketIdentifier getBucketIdentifier(String partition) {
+ HoodieSparkConsistentBucketIndex index =
(HoodieSparkConsistentBucketIndex) table.getIndex();
+ HoodieConsistentHashingMetadata metadata =
ConsistentBucketIndexUtils.loadOrCreateMetadata(this.table, partition,
index.getNumBuckets());
+ if (hashingChildrenNodes.containsKey(partition)) {
+ metadata.setChildrenNodes(hashingChildrenNodes.get(partition));
+ }
+ return new ConsistentBucketIdentifier(metadata);
+ }
+
+ @Override
+ public Dataset<Row> repartitionRecords(Dataset<Row> rows, int
outputPartitions) {
+ JavaRDD<Row> rowJavaRDD = rows.toJavaRDD();
+ prepareRepartition(rowJavaRDD);
+ Partitioner partitioner = new Partitioner() {
+ @Override
+ public int getPartition(Object key) {
+ return (int) key;
+ }
+
+ @Override
+ public int numPartitions() {
+ return fileIdPfxList.size();
+ }
+ };
+
+ return rows.sparkSession().createDataFrame(rowJavaRDD
+ .mapToPair(row -> new Tuple2<>(getBucketId(row), row))
+ .partitionBy(partitioner)
+ .values(), rows.schema());
+ }
+
+ /**
+ * Prepare consistent hashing metadata for repartition
+ *
+ * @param rows input records
+ */
+ private void prepareRepartition(JavaRDD<Row> rows) {
+ this.partitionToIdentifier = initializeBucketIdentifier(rows);
+ this.partitionToFileIdPfxIdxMap =
ConsistentBucketIndexUtils.generatePartitionToFileIdPfxIdxMap(partitionToIdentifier);
+ partitionToIdentifier.values().forEach(identifier -> {
+
fileIdPfxList.addAll(identifier.getNodes().stream().map(ConsistentHashingNode::getFileIdPrefix).collect(Collectors.toList()));
+ });
+ }
+
+ /**
+ * Initialize hashing metadata of input records. The metadata of all related
partitions will be loaded, and
+ * the mapping from partition to its bucket identifier is constructed.
+ */
+ private Map<String, ConsistentBucketIdentifier>
initializeBucketIdentifier(JavaRDD<Row> rows) {
+ return
rows.map(this.extractor::getPartitionPath).distinct().collect().stream()
+ .collect(Collectors.toMap(p -> p, this::getBucketIdentifier));
+ }
+
+ @Override
+ public void addHashingChildrenNodes(String partition,
List<ConsistentHashingNode> nodes) {
+ ValidationUtils.checkState(nodes.stream().noneMatch(n -> n.getTag() ==
ConsistentHashingNode.NodeTag.NORMAL),
Review Comment:
`Normal` nodes are read from metadata in file system, while `REPLACE` and
`DELETE` nodes are read from consistent hashing resizing plan. Here we are
adding nodes read from clustering plan to the hash metadata so all nodes must
not be `NORMAL`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]