From 6bb8dd66bd203c027d51efdc4da04562bec0f166 Mon Sep 17 00:00:00 2001 From: Rex An Date: Tue, 5 Jul 2022 09:52:47 +0800 Subject: [PATCH 01/18] Enable row writer for clustering --- .../client/HoodieInternalWriteStatus.java | 8 + .../HoodieInternalWriteStatusCoordinator.java | 55 +++++ .../hudi/config/HoodieClusteringConfig.java | 7 + .../apache/hudi/config/HoodieWriteConfig.java | 31 ++- .../MultipleSparkJobExecutionStrategy.java | 201 +++++++++++++++++- .../SparkSingleFileSortExecutionStrategy.java | 17 ++ .../SparkSortAndSizeExecutionStrategy.java | 16 ++ .../RowCustomColumnsSortPartitioner.java | 4 + .../RowSpatialCurveSortPartitioner.java | 75 +++++++ .../io/storage/row/HoodieRowCreateHandle.java | 25 ++- .../org/apache/hudi/common/fs/FSUtils.java | 4 + .../hudi/common/model/HoodieRecord.java | 4 + .../hudi/HoodieDatasetBulkInsertHelper.java | 0 .../DataSourceInternalWriterHelper.java | 5 +- .../scala/org/apache/hudi/DefaultSource.scala | 10 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 32 ++- .../HoodieDataSourceInternalWriter.java | 26 ++- .../HoodieDataSourceInternalBatchWrite.java | 27 ++- 18 files changed, 504 insertions(+), 43 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java create mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java index 815ef4892e68f..5a2449615121a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java @@ -149,6 +149,14 @@ public void setSuccessRecordKeys(List successRecordKeys) { this.successRecordKeys = successRecordKeys; } + public double getFailureFraction() { + return failureFraction; + } + + public boolean isTrackSuccessRecords() { + return trackSuccessRecords; + } + @Override public String toString() { return "PartitionPath " + partitionPath + ", FileID " + fileId + ", Success records " diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java new file mode 100644 index 0000000000000..1111454de04e9 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java @@ -0,0 +1,55 @@ +/* + * 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.client; + +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +public class HoodieInternalWriteStatusCoordinator { + private static final HoodieInternalWriteStatusCoordinator INSTANCE = new HoodieInternalWriteStatusCoordinator(); + private final ConcurrentHashMap> statusesMap; + private HoodieInternalWriteStatusCoordinator() { + this.statusesMap = new ConcurrentHashMap<>(); + } + + public static HoodieInternalWriteStatusCoordinator get() { + return INSTANCE; + } + + /** + * Call this to save writeStatuses corresponding with an identifyId. + */ + public void assignStatuses(String identifyId, List writeStatuses) { + statusesMap.put(identifyId, writeStatuses); + } + + /** + * Return related internalWriteStatuses by the identifyId. + */ + public List getWriteStatuses(String identifyId) { + return statusesMap.get(identifyId); + } + + /** + * Remove related internalWriteStatuses by the identifyId. + */ + public void removeStatuses(String identifyId) { + statusesMap.remove(identifyId); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index 1180845a6ed8a..bb90132ada1a2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -138,6 +138,13 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.9.0") .withDocumentation("Config to control frequency of async clustering"); + public static final ConfigProperty CLUSTERING_AS_ROW = ConfigProperty + .key("hoodie.clustering.as.row") + .defaultValue(false) + .sinceVersion("0.12.0") + .withDocumentation("If set to true, will perform bulk_insert operation directly using the spark native " + + "`Row` representation, avoiding any additional conversion costs."); + public static final ConfigProperty PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions") .defaultValue("0") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6178e63e3606c..d4e3c2692b584 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -211,6 +211,23 @@ public class HoodieWriteConfig extends HoodieConfig { + " optimally for common query patterns. For now we support a build-in user defined bulkinsert partitioner org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner" + " which can does sorting based on specified column values set by " + BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS.key()); + public static final ConfigProperty BULKINSERT_ROW_IDENTIFY_ID = ConfigProperty + .key("hoodie.bulkinsert.row.writestatus.id") + .noDefaultValue() + .withDocumentation("The unique id for each write operation, HoodieInternalWriteStatusCoordinator will use " + + "this id to identify the related write statuses"); + public static final ConfigProperty BULKINSERT_PRESERVE_METADATA = ConfigProperty + .key("hoodie.datasource.preserve.metadata") + .defaultValue(false) + .withDocumentation("Whether to preserve metadata from the read source, this is only take effective " + + "When using bulk_insert and row writer is enabled"); + + public static final ConfigProperty BULKINSERT_ROW_AUTO_COMMIT = ConfigProperty + .key("hoodie.bulkinsert.row.auto.commit") + .defaultValue(true) + .withDocumentation("Whether to create request, inflight and post commit automatically, this can be turned " + + "off to perform inspection of the uncommitted write before deciding to commit."); + public static final ConfigProperty UPSERT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.upsert.shuffle.parallelism") .defaultValue("200") @@ -319,15 +336,15 @@ public class HoodieWriteConfig extends HoodieConfig { + "lowest and best effort file sizing. " + "NONE: No sorting. Fastest and matches `spark.write.parquet()` in terms of number of files, overheads"); - public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_ENABLE = ConfigProperty + public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_ENABLE = ConfigProperty .key("hoodie.embed.timeline.server") - .defaultValue("true") + .defaultValue(true) .withDocumentation("When true, spins up an instance of the timeline server (meta server that serves cached file listings, statistics)," + "running on each writer's driver process, accepting requests during the write from executors."); - public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED = ConfigProperty + public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED = ConfigProperty .key("hoodie.embed.timeline.server.reuse.enabled") - .defaultValue("false") + .defaultValue(false) .withDocumentation("Controls whether the timeline server instance should be cached and reused across the JVM (across task lifecycles)" + "to avoid startup costs. This should rarely be changed."); @@ -982,6 +999,10 @@ public Boolean shouldAutoCommit() { return getBoolean(AUTO_COMMIT_ENABLE); } + public Boolean bulkInsertRowAutoCommit() { + return getBoolean(BULKINSERT_ROW_AUTO_COMMIT); + } + public Boolean shouldAssumeDatePartitioning() { return metadataConfig.shouldAssumeDatePartitioning(); } @@ -1084,7 +1105,7 @@ public boolean isEmbeddedTimelineServerEnabled() { } public boolean isEmbeddedTimelineServerReuseEnabled() { - return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED)); + return getBoolean(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED); } public int getEmbeddedTimelineServerPort() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index df0ad6e2b82da..3d4a64b891a86 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.HoodieInternalWriteStatusCoordinator; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -42,6 +43,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; @@ -49,11 +51,15 @@ import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.execution.bulkinsert.RDDSpatialCurveSortPartitioner; +import org.apache.hudi.execution.bulkinsert.RowCustomColumnsSortPartitioner; +import org.apache.hudi.execution.bulkinsert.RowSpatialCurveSortPartitioner; import org.apache.hudi.io.IOUtils; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenUtils; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; @@ -68,12 +74,20 @@ 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 org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SaveMode; +import scala.collection.JavaConverters; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -98,10 +112,18 @@ public HoodieWriteMetadata> performClustering(final Hood // execute clustering for each group async and collect WriteStatus Stream> writeStatusesStream = FutureUtils.allOf( clusteringPlan.getInputGroups().stream() - .map(inputGroup -> runClusteringForGroupAsync(inputGroup, - clusteringPlan.getStrategy().getStrategyParams(), - Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), - instantTime)) + .map(inputGroup -> { + if (Boolean.parseBoolean(getWriteConfig().getString(HoodieClusteringConfig.CLUSTERING_AS_ROW))) { + return runClusteringForGroupAsyncWithRow(inputGroup, + clusteringPlan.getStrategy().getStrategyParams(), + Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), + instantTime); + } + return runClusteringForGroupAsyncWithRDD(inputGroup, + clusteringPlan.getStrategy().getStrategyParams(), + Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), + instantTime); + }) .collect(Collectors.toList())) .join() .stream(); @@ -113,6 +135,16 @@ public HoodieWriteMetadata> performClustering(final Hood return writeMetadata; } + /** + * Execute clustering to write inputRecords into new files based on strategyParams. + * Different from {@link performClusteringWithRecordsRDD}, this method take {@link Dataset} + * as inputs. + */ + public abstract HoodieData performClusteringWithRecordsRow(final Dataset inputRecords, final int numOutputGroups, final String instantTime, + final Map strategyParams, final Schema schema, + final List fileGroupIdList, final boolean preserveHoodieMetadata); + + /** * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. * The number of new file groups created is bounded by numOutputGroups. @@ -131,6 +163,57 @@ public abstract HoodieData performClusteringWithRecordsRDD(final Ho final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata); + + protected HoodieData performRowWrite(Dataset inputRecords, Map parameters) { + String uuid = UUID.randomUUID().toString(); + parameters.put(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key(), uuid); + try { + inputRecords.write() + .format("hudi") + .options(JavaConverters.mapAsScalaMapConverter(parameters).asScala()) + .mode(SaveMode.Append) + .save(getWriteConfig().getBasePath()); + List writeStatusList = HoodieInternalWriteStatusCoordinator.get().getWriteStatuses(uuid) + .stream() + .map(internalWriteStatus -> { + WriteStatus status = new WriteStatus( + internalWriteStatus.isTrackSuccessRecords(), internalWriteStatus.getFailureFraction()); + status.setFileId(internalWriteStatus.getFileId()); + status.setTotalRecords(internalWriteStatus.getTotalRecords()); + status.setPartitionPath(internalWriteStatus.getPartitionPath()); + status.setStat(internalWriteStatus.getStat()); + return status; + }).collect(Collectors.toList()); + return getEngineContext().parallelize(writeStatusList); + } finally { + HoodieInternalWriteStatusCoordinator.get().removeStatuses(uuid); + } + } + + protected Map buildHoodieRowParameters(int numOutputGroups, String instantTime, Map strategyParams, boolean preserveHoodieMetadata) { + HashMap params = new HashMap<>(); + HoodieWriteConfig writeConfig = getWriteConfig(); + params.put(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key(), String.valueOf(numOutputGroups)); + params.put(HoodieWriteConfig.BULKINSERT_ROW_AUTO_COMMIT.key(), String.valueOf(false)); + params.put(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED.key(), String.valueOf(true)); + if (writeConfig.populateMetaFields()) { + // If the table enables populateMetaFields, use SimpleKeyGenerator with the metadata of + // recordKey and partitionPath to avoid regeneration. + params.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName()); + params.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), HoodieRecord.RECORD_KEY_METADATA_FIELD); + params.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), HoodieRecord.PARTITION_PATH_METADATA_FIELD); + } else { + params.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), writeConfig.getKeyGeneratorClass()); + params.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), writeConfig.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); + params.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), writeConfig.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)); + } + params.put("hoodie.datasource.write.operation", "bulk_insert"); + params.put("hoodie.instant.time", instantTime); + params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), String.valueOf(preserveHoodieMetadata)); + configRowPartitioner(strategyParams, params); + return params; + } + /** * Create {@link BulkInsertPartitioner} based on strategy params. * @@ -164,13 +247,41 @@ protected BulkInsertPartitioner>> getPartitioner(Map>} based on strategy params. + * + * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. + * @param parameters Map used to write the partitioner configurations. + * @return {@link RDDCustomColumnsSortPartitioner} if sort columns are provided, otherwise empty. */ - private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams, - boolean preserveHoodieMetadata, String instantTime) { + protected void configRowPartitioner(Map strategyParams, Map parameters) { + Option.ofNullable(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key())) + .map(orderColumnStr -> { + parameters.put(HoodieWriteConfig.BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS.key(), orderColumnStr); + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy = getWriteConfig().getLayoutOptimizationStrategy(); + switch (layoutOptStrategy) { + case ZORDER: + case HILBERT: + parameters.put(HoodieWriteConfig.BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME.key(), RowSpatialCurveSortPartitioner.class.getCanonicalName()); + break; + case LINEAR: + parameters.put(HoodieWriteConfig.BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME.key(), RowCustomColumnsSortPartitioner.class.getCanonicalName()); + break; + default: + throw new UnsupportedOperationException(String.format("Layout optimization strategy '%s' is not supported", layoutOptStrategy)); + } + return orderColumnStr; + }); + } + + + /** + * Submit job to execute clustering for the group with RDD APIs. + */ + private CompletableFuture> runClusteringForGroupAsyncWithRDD(HoodieClusteringGroup clusteringGroup, Map strategyParams, + boolean preserveHoodieMetadata, String instantTime) { return CompletableFuture.supplyAsync(() -> { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); - HoodieData> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); + HoodieData> inputRecords = readRecordsForGroupAsRDD(jsc, clusteringGroup, instantTime); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) @@ -179,10 +290,26 @@ private CompletableFuture> runClusteringForGroupAsync(Ho }); } + /** + * Submit job to execute clustering for the group with dataset APIs. + */ + private CompletableFuture> runClusteringForGroupAsyncWithRow(HoodieClusteringGroup clusteringGroup, Map strategyParams, + boolean preserveHoodieMetadata, String instantTime) { + return CompletableFuture.supplyAsync(() -> { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); + Dataset inputRecords = readRecordsForGroupAsRow(jsc, clusteringGroup, instantTime); + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); + List inputFileIds = clusteringGroup.getSlices().stream() + .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) + .collect(Collectors.toList()); + return performClusteringWithRecordsRow(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, preserveHoodieMetadata); + }); + } + /** * Get RDD of all records for the group. This includes all records from file slice (Apply updates from log files, if any). */ - private HoodieData> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { + private HoodieData> readRecordsForGroupAsRDD(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); if (hasLogFiles) { @@ -273,6 +400,62 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex .map(record -> transform(record, writeConfig))); } + /** + * Get dataset of all records for the group. This includes all records from file slice (Apply updates from log files, if any). + */ + private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, + HoodieClusteringGroup clusteringGroup, + String instantTime) { + List clusteringOps = clusteringGroup.getSlices().stream() + .map(ClusteringOperation::create).collect(Collectors.toList()); + boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); + SQLContext sqlContext = new SQLContext(jsc.sc()); + + String[] baseFilePaths = clusteringOps + .stream() + .map(op -> { + ArrayList pairs = new ArrayList<>(); + if (op.getBootstrapFilePath() != null) { + pairs.add(op.getBootstrapFilePath()); + } + if (op.getDataFilePath() != null) { + pairs.add(op.getDataFilePath()); + } + return pairs; + }) + .flatMap(Collection::stream) + .filter(path -> !path.isEmpty()) + .toArray(String[]::new); + String[] deltaPaths = clusteringOps + .stream() + .filter(op -> !op.getDeltaFilePaths().isEmpty()) + .flatMap(op -> op.getDeltaFilePaths().stream()) + .toArray(String[]::new); + + Dataset inputRecords; + if (hasLogFiles) { + String compactionFractor = Option.ofNullable(getWriteConfig().getString("compaction.memory.fraction")) + .orElse("0.75"); + String[] paths = new String[baseFilePaths.length + deltaPaths.length]; + System.arraycopy(baseFilePaths, 0, paths, 0, baseFilePaths.length); + System.arraycopy(deltaPaths, 0, paths, baseFilePaths.length, deltaPaths.length); + inputRecords = sqlContext.read() + .format("org.apache.hudi") + .option("hoodie.datasource.query.type", "snapshot") + .option("compaction.memory.fraction", compactionFractor) + .option("as.of.instant", instantTime) + .option("hoodie.datasource.read.paths", String.join(",", paths)) + .load(); + } else { + inputRecords = sqlContext.read() + .format("org.apache.hudi") + .option("as.of.instant", instantTime) + .option("hoodie.datasource.read.paths", String.join(",", baseFilePaths)) + .load(); + } + return inputRecords; + } + /** * Stream to array conversion with generic type is not straightforward. * Implement a utility method to abstract high level logic. This needs to be improved in future diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index b61017c34ce41..d98e5b554b515 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -35,7 +35,10 @@ 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.HashMap; import java.util.List; import java.util.Map; @@ -54,6 +57,20 @@ public SparkSingleFileSortExecutionStrategy(HoodieTable table, super(table, engineContext, writeConfig); } + @Override + public HoodieData performClusteringWithRecordsRow(Dataset inputRecords, int numOutputGroups, String instantTime, Map strategyParams, Schema schema, List fileGroupIdList, boolean preserveHoodieMetadata) { + 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); + + HashMap params = new HashMap<>( + buildHoodieRowParameters(numOutputGroups, instantTime, strategyParams, preserveHoodieMetadata)); + params.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(Long.MAX_VALUE)); + + return performRowWrite(inputRecords, params); + } + @Override public HoodieData performClusteringWithRecordsRDD(HoodieData> inputRecords, int numOutputGroups, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 7db63d4169fc1..7f5120c9d8e42 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -33,7 +33,10 @@ 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.HashMap; import java.util.List; import java.util.Map; @@ -52,6 +55,19 @@ public SparkSortAndSizeExecutionStrategy(HoodieTable table, super(table, engineContext, writeConfig); } + @Override + public HoodieData performClusteringWithRecordsRow(Dataset inputRecords, int numOutputGroups, + String instantTime, Map strategyParams, Schema schema, + List fileGroupIdList, boolean preserveHoodieMetadata) { + LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); + + HashMap params = new HashMap<>( + buildHoodieRowParameters(numOutputGroups, instantTime, strategyParams, preserveHoodieMetadata)); + params.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); + + return performRowWrite(inputRecords, params); + } + @Override public HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java index ceeb2b3fe8f00..324c462cbb947 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java @@ -53,6 +53,10 @@ public boolean arePartitionRecordsSorted() { return true; } + public String[] getSortColumnNames() { + return sortColumnNames; + } + private String[] getSortColumnName(HoodieWriteConfig config) { return Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(",")) .map(String::trim).toArray(String[]::new); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java new file mode 100644 index 0000000000000..b3f69c3ae6f86 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java @@ -0,0 +1,75 @@ +/* + * 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.hudi.sort.SpaceCurveSortingHelper; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import java.util.Arrays; +import java.util.List; + +public class RowSpatialCurveSortPartitioner extends RowCustomColumnsSortPartitioner { + + private final String[] orderByColumns; + private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy; + private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType; + + public RowSpatialCurveSortPartitioner(HoodieWriteConfig config) { + super(config); + this.layoutOptStrategy = config.getLayoutOptimizationStrategy(); + if (config.getClusteringSortColumns() != null) { + this.orderByColumns = Arrays.stream(config.getClusteringSortColumns().split(",")) + .map(String::trim).toArray(String[]::new); + } else { + this.orderByColumns = getSortColumnNames(); + } + this.curveCompositionStrategyType = config.getLayoutOptimizationCurveBuildMethod(); + } + + @Override + public Dataset repartitionRecords(Dataset records, int outputPartitions) { + return reorder(records, outputPartitions); + } + + private Dataset reorder(Dataset dataset, int numOutputGroups) { + if (orderByColumns.length == 0) { + // No-op + return dataset; + } + + List 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; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index e7c6ccd6fadb1..56a718ef6da0b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -70,6 +70,8 @@ public class HoodieRowCreateHandle implements Serializable { private final UTF8String commitTime; private final Function seqIdGenerator; + private final boolean preserveMetadata; + private final HoodieTimer currTimer; protected final HoodieInternalRowFileWriter fileWriter; @@ -104,6 +106,8 @@ public HoodieRowCreateHandle(HoodieTable table, this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(), writeConfig.getWriteStatusFailureFraction()); + this.preserveMetadata = writeConfig.getBoolean(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA); + writeStatus.setPartitionPath(partitionPath); writeStatus.setFileId(fileId); writeStatus.setStat(new HoodieWriteStat()); @@ -153,13 +157,22 @@ private void writeRow(InternalRow row) { // over again) UTF8String recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); - // This is the only meta-field that is generated dynamically, hence conversion b/w - // [[String]] and [[UTF8String]] is unavoidable - UTF8String seqId = UTF8String.fromString(seqIdGenerator.apply(GLOBAL_SEQ_NO.getAndIncrement())); - - InternalRow updatedRow = new HoodieInternalRow(commitTime, seqId, recordKey, - partitionPath, fileName, row, true); + InternalRow updatedRow; + if (preserveMetadata) { + updatedRow = new HoodieInternalRow(row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_POS), + row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_POS), + recordKey, + partitionPath, + fileName, row, true); + } else { + // This is the only meta-field that is generated dynamically, hence conversion b/w + // [[String]] and [[UTF8String]] is unavoidable + UTF8String seqId = UTF8String.fromString(seqIdGenerator.apply(GLOBAL_SEQ_NO.getAndIncrement())); + + updatedRow = new HoodieInternalRow(commitTime, seqId, recordKey, + partitionPath, fileName, row, true); + } try { fileWriter.writeRow(recordKey, updatedRow); // NOTE: To avoid conversion on the hot-path we only convert [[UTF8String]] into [[String]] diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index d940f3bb4577a..1b60856519689 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -754,6 +754,10 @@ public static Map parallelizeFilesProcess( if (subPaths.size() > 0) { SerializableConfiguration conf = new SerializableConfiguration(fs.getConf()); int actualParallelism = Math.min(subPaths.size(), parallelism); + + hoodieEngineContext.setJobStatus(FSUtils.class.getSimpleName(), + "Parallel listing paths " + String.join(",", subPaths)); + result = hoodieEngineContext.mapToPair(subPaths, subPath -> new ImmutablePair<>(subPath, pairFunction.apply(new ImmutablePair<>(subPath, conf))), actualParallelism); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index c7ef08a16264c..6d6ae7ffdbbba 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -61,6 +61,10 @@ public abstract class HoodieRecord implements Serializable { public static int PARTITION_PATH_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD); public static int FILENAME_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD); + public static int COMMIT_TIME_METADATA_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_TIME_METADATA_FIELD); + + public static int COMMIT_SEQNO_METADATA_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_SEQNO_METADATA_FIELD); + /** * Identifies the record across the table. */ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index 3a349473b2201..0de0cf7c7c575 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -63,7 +63,6 @@ public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writ this.extraMetadata = extraMetadata; this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig); writeClient.setOperationType(operationType); - writeClient.startCommitWithTime(instantTime); this.metaClient = HoodieTableMetaClient.builder().setConf(configuration).setBasePath(writeConfig.getBasePath()).build(); this.metaClient.validateTableProperties(writeConfig.getProps()); @@ -96,6 +95,10 @@ public void abort() { writeClient.close(); } + public void createRequestedCommit() { + writeClient.startCommitWithTime(instantTime); + } + public void createInflightCommit() { metaClient.getActiveTimeline().transitionRequestedToInflight( new HoodieInstant(State.REQUESTED, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index d5cbf020ed3e5..38b418efd857f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -175,14 +175,16 @@ class DefaultSource extends RelationProvider mode: SaveMode, optParams: Map[String, String], df: DataFrame): BaseRelation = { - val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) + val colsWithoutHoodieMeta = df.schema + .filter(field => !HoodieRecord.HOODIE_META_COLUMNS.contains(field.name)) + .foldLeft(new StructType())((s, f) => s.add(f)) if (optParams.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)) { - HoodieSparkSqlWriter.bootstrap(sqlContext, mode, optParams, dfWithoutMetaCols) + HoodieSparkSqlWriter.bootstrap(sqlContext, mode, optParams, df) } else { - HoodieSparkSqlWriter.write(sqlContext, mode, optParams, dfWithoutMetaCols) + HoodieSparkSqlWriter.write(sqlContext, mode, optParams, df) } - new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema) + new HoodieEmptyRelation(sqlContext, colsWithoutHoodieMeta) } override def createSink(sqlContext: SQLContext, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index c07024810017a..6f75c9ac9d1a3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -124,7 +124,7 @@ object HoodieSparkSqlWriter { jsc.setLocalProperty("spark.scheduler.pool", SparkConfigs.SPARK_DATASOURCE_WRITER_POOL_NAME) } } - val instantTime = HoodieActiveTimeline.createNewInstantTime() + val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(hoodieConfig.getProps)) if (mode == SaveMode.Ignore && tableExists) { @@ -170,6 +170,11 @@ object HoodieSparkSqlWriter { val commitActionType = CommitUtils.getCommitActionType(operation, tableConfig.getTableType) val dropPartitionColumns = hoodieConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS) + val instantTime = Option(hoodieConfig.getString(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY)) match { + case Some(instant) => instant + case _ => HoodieActiveTimeline.createNewInstantTime() + } + // short-circuit if bulk_insert via row is enabled. // scalastyle:off if (hoodieConfig.getBoolean(ENABLE_ROW_WRITER) && @@ -180,11 +185,13 @@ object HoodieSparkSqlWriter { } // scalastyle:on + val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS:_*) + val reconcileSchema = parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) = operation match { case WriteOperationType.DELETE => { - val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) + val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, dfWithoutMetaCols, reconcileSchema) // Convert to RDD[HoodieKey] val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD() @@ -221,7 +228,7 @@ object HoodieSparkSqlWriter { val partitionColsToDelete = parameters(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key()).split(",") java.util.Arrays.asList(partitionColsToDelete: _*) } else { - val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) + val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, dfWithoutMetaCols, reconcileSchema) genericRecords.map(gr => keyGenerator.getKey(gr).getPartitionPath).toJavaRDD().distinct().collect() } // Create a HoodieWriteClient & issue the delete. @@ -242,7 +249,7 @@ object HoodieSparkSqlWriter { classOf[org.apache.avro.Schema])) // TODO(HUDI-4472) revisit and simplify schema handling - val sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) + val sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dfWithoutMetaCols.schema, structName, nameSpace) val latestTableSchema = getLatestTableSchema(fs, basePath, sparkContext).getOrElse(sourceSchema) val schemaEvolutionEnabled = parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean @@ -333,7 +340,7 @@ object HoodieSparkSqlWriter { // Check for errors and commit the write. val (writeSuccessful, compactionInstant, clusteringInstant) = - commitAndPerformPostOperations(sqlContext.sparkSession, df.schema, + commitAndPerformPostOperations(sqlContext.sparkSession, dfWithoutMetaCols.schema, writeResult, parameters, writeClient, tableConfig, jsc, TableInstantInfo(basePath, instantTime, commitActionType, operation)) @@ -427,6 +434,7 @@ object HoodieSparkSqlWriter { df: DataFrame, hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty, hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty): Boolean = { + val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS:_*) assert(optParams.get("path").exists(!StringUtils.isNullOrEmpty(_)), "'path' must be set") val path = optParams("path") @@ -446,9 +454,9 @@ object HoodieSparkSqlWriter { val bootstrapIndexClass = hoodieConfig.getStringOrDefault(INDEX_CLASS_NAME) var schema: String = null - if (df.schema.nonEmpty) { + if (dfWithoutMetaCols.schema.nonEmpty) { val (structName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName) - schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, namespace).toString + schema = AvroConversionUtils.convertStructTypeToAvroSchema(dfWithoutMetaCols.schema, structName, namespace).toString } else { schema = HoodieAvroUtils.getNullSchema.toString } @@ -506,7 +514,7 @@ object HoodieSparkSqlWriter { } finally { writeClient.close() } - val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema) + val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, dfWithoutMetaCols.schema) metaSyncSuccess } } @@ -533,11 +541,15 @@ object HoodieSparkSqlWriter { val dropPartitionColumns = parameters.get(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key()).map(_.toBoolean) .getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()) // register classes & schemas + val colsWithoutHoodieMeta = df.schema + .filter(field => !HoodieRecord.HOODIE_META_COLUMNS.contains(field.name)) + .foldLeft(new StructType())((s, f) => s.add(f)) + val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName) sparkContext.getConf.registerKryoClasses( Array(classOf[org.apache.avro.generic.GenericData], classOf[org.apache.avro.Schema])) - var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) + var schema = AvroConversionUtils.convertStructTypeToAvroSchema(colsWithoutHoodieMeta, structName, nameSpace) if (dropPartitionColumns) { schema = generateSchemaWithoutPartitionColumns(partitionColumns, schema) } @@ -589,7 +601,7 @@ object HoodieSparkSqlWriter { throw new HoodieException("Bulk insert using row writer is not supported with current Spark version." + " To use row writer please switch to spark 2 or spark 3") } - val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, df.schema) + val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, colsWithoutHoodieMeta) (syncHiveSuccess, common.util.Option.ofNullable(instantTime)) } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java index c4b21483e8fee..a7e26d4bc77fa 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -20,8 +20,10 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.client.HoodieInternalWriteStatusCoordinator; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hadoop.conf.Configuration; @@ -64,11 +66,17 @@ public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writ this.extraMetadataMap = DataSourceUtils.getExtraMetadata(dataSourceOptions.asMap()); this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, sparkSession, configuration, extraMetadataMap); + + if (writeConfig.bulkInsertRowAutoCommit()) { + this.dataSourceInternalWriterHelper.createRequestedCommit(); + } } @Override public DataWriterFactory createWriterFactory() { - dataSourceInternalWriterHelper.createInflightCommit(); + if (writeConfig.bulkInsertRowAutoCommit()) { + dataSourceInternalWriterHelper.createInflightCommit(); + } if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted); @@ -89,9 +97,19 @@ public void onDataWriterCommit(WriterCommitMessage message) { @Override public void commit(WriterCommitMessage[] messages) { - List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) - .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); - dataSourceInternalWriterHelper.commit(writeStatList); + List writeStatusList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream()) + .collect(Collectors.toList()); + if (writeConfig.bulkInsertRowAutoCommit()) { + List writeStatList = writeStatusList.stream() + .map(HoodieInternalWriteStatus::getStat).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); + } + + Option.ofNullable(writeConfig.getString(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key())).map(id -> { + HoodieInternalWriteStatusCoordinator.get().assignStatuses(id, writeStatusList); + return true; + }); } @Override diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java index fb5f609d79ef2..ed7072546389a 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java @@ -20,8 +20,10 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.client.HoodieInternalWriteStatusCoordinator; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.internal.DataSourceInternalWriterHelper; @@ -63,11 +65,18 @@ public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig this.extraMetadata = DataSourceUtils.getExtraMetadata(properties); this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, jss, hadoopConfiguration, extraMetadata); + + if (writeConfig.bulkInsertRowAutoCommit()) { + this.dataSourceInternalWriterHelper.createRequestedCommit(); + } } @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - dataSourceInternalWriterHelper.createInflightCommit(); + if (writeConfig.shouldAutoCommit()) { + dataSourceInternalWriterHelper.createInflightCommit(); + } + if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted); @@ -88,9 +97,19 @@ public void onDataWriterCommit(WriterCommitMessage message) { @Override public void commit(WriterCommitMessage[] messages) { - List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) - .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); - dataSourceInternalWriterHelper.commit(writeStatList); + List writeStatusList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream()) + .collect(Collectors.toList()); + if (writeConfig.shouldAutoCommit()) { + List writeStatList = writeStatusList.stream() + .map(HoodieInternalWriteStatus::getStat).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); + } + + Option.ofNullable(writeConfig.getString(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key())).map(id -> { + HoodieInternalWriteStatusCoordinator.get().assignStatuses(id, writeStatusList); + return true; + }); } @Override From d94e3ecc442323f2bd33e1228ebdf40987c46013 Mon Sep 17 00:00:00 2001 From: Rex An Date: Tue, 5 Jul 2022 17:01:25 +0800 Subject: [PATCH 02/18] Fix bugs --- .../hudi/config/HoodieClusteringConfig.java | 4 +- .../apache/hudi/config/HoodieWriteConfig.java | 4 +- .../MultipleSparkJobExecutionStrategy.java | 21 +++---- .../SparkSingleFileSortExecutionStrategy.java | 8 ++- ...HoodieSparkMergeOnReadTableClustering.java | 59 +++++++++++++------ .../functional/TestLayoutOptimization.scala | 42 ++++++++----- 6 files changed, 87 insertions(+), 51 deletions(-) rename {hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table => hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi}/functional/TestHoodieSparkMergeOnReadTableClustering.java (86%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index bb90132ada1a2..d1a8fc03f7755 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -142,8 +142,8 @@ public class HoodieClusteringConfig extends HoodieConfig { .key("hoodie.clustering.as.row") .defaultValue(false) .sinceVersion("0.12.0") - .withDocumentation("If set to true, will perform bulk_insert operation directly using the spark native " + - "`Row` representation, avoiding any additional conversion costs."); + .withDocumentation("If set to true, will perform bulk_insert operation directly using the spark native " + + "`Row` representation, avoiding any additional conversion costs."); public static final ConfigProperty PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index d4e3c2692b584..63d2a29431e2b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -336,9 +336,9 @@ public class HoodieWriteConfig extends HoodieConfig { + "lowest and best effort file sizing. " + "NONE: No sorting. Fastest and matches `spark.write.parquet()` in terms of number of files, overheads"); - public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_ENABLE = ConfigProperty + public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_ENABLE = ConfigProperty .key("hoodie.embed.timeline.server") - .defaultValue(true) + .defaultValue("true") .withDocumentation("When true, spins up an instance of the timeline server (meta server that serves cached file listings, statistics)," + "running on each writer's driver process, accepting requests during the write from executors."); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 3d4a64b891a86..b16bf4c039535 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -43,7 +43,6 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; @@ -58,7 +57,6 @@ import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenUtils; -import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.BulkInsertPartitioner; @@ -163,7 +161,6 @@ public abstract HoodieData performClusteringWithRecordsRDD(final Ho final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata); - protected HoodieData performRowWrite(Dataset inputRecords, Map parameters) { String uuid = UUID.randomUUID().toString(); parameters.put(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key(), uuid); @@ -196,17 +193,9 @@ protected Map buildHoodieRowParameters(int numOutputGroups, Stri params.put(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key(), String.valueOf(numOutputGroups)); params.put(HoodieWriteConfig.BULKINSERT_ROW_AUTO_COMMIT.key(), String.valueOf(false)); params.put(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED.key(), String.valueOf(true)); - if (writeConfig.populateMetaFields()) { - // If the table enables populateMetaFields, use SimpleKeyGenerator with the metadata of - // recordKey and partitionPath to avoid regeneration. - params.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName()); - params.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), HoodieRecord.RECORD_KEY_METADATA_FIELD); - params.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), HoodieRecord.PARTITION_PATH_METADATA_FIELD); - } else { - params.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), writeConfig.getKeyGeneratorClass()); - params.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), writeConfig.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); - params.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), writeConfig.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)); - } + params.compute(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), (k, v) -> writeConfig.getKeyGeneratorClass()); + params.compute(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), (k, v) -> writeConfig.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); + params.compute(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), (k, v) -> writeConfig.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)); params.put("hoodie.datasource.write.operation", "bulk_insert"); params.put("hoodie.instant.time", instantTime); params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), String.valueOf(preserveHoodieMetadata)); @@ -269,6 +258,10 @@ protected void configRowPartitioner(Map strategyParams, Map getWriteConfig().getString(HoodieClusteringConfig.LAYOUT_OPTIMIZE_STRATEGY)); + parameters.compute(HoodieClusteringConfig.LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD.key(), + (k, v) -> getWriteConfig().getString(HoodieClusteringConfig.LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD)); return orderColumnStr; }); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index d98e5b554b515..c95d5398f2696 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -58,7 +58,13 @@ public SparkSingleFileSortExecutionStrategy(HoodieTable table, } @Override - public HoodieData performClusteringWithRecordsRow(Dataset inputRecords, int numOutputGroups, String instantTime, Map strategyParams, Schema schema, List fileGroupIdList, boolean preserveHoodieMetadata) { + public HoodieData performClusteringWithRecordsRow(Dataset inputRecords, + int numOutputGroups, + String instantTime, + Map strategyParams, + Schema schema, + List fileGroupIdList, + boolean preserveHoodieMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java similarity index 86% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java index 5438fbcfc0d98..c10d17f346471 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.hudi.table.functional; +package org.apache.hudi.functional; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.model.HoodieBaseFile; @@ -45,8 +45,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.provider.ValueSource; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Stream; @@ -60,20 +60,30 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes private static Stream testClustering() { return Stream.of( - Arguments.of(true, true, true), - Arguments.of(true, true, false), - Arguments.of(true, false, true), - Arguments.of(true, false, false), - Arguments.of(false, true, true), - Arguments.of(false, true, false), - Arguments.of(false, false, true), - Arguments.of(false, false, false) - ); + Arrays.asList(true, true, true), + Arrays.asList(true, true, false), + Arrays.asList(true, false, true), + Arrays.asList(true, false, false), + Arrays.asList(false, true, true), + Arrays.asList(false, true, false), + Arrays.asList(false, false, true), + Arrays.asList(false, false, false)) + .flatMap(arguments -> { + ArrayList enableRowClusteringArgs = new ArrayList<>(); + enableRowClusteringArgs.add(true); + enableRowClusteringArgs.addAll(arguments); + ArrayList disableRowClusteringArgs = new ArrayList<>(); + disableRowClusteringArgs.add(false); + disableRowClusteringArgs.addAll(arguments); + return Stream.of( + Arguments.of(enableRowClusteringArgs.toArray(new Boolean[0])), + Arguments.of(disableRowClusteringArgs.toArray(new Boolean[0]))); + }); } @ParameterizedTest @MethodSource - void testClustering(boolean doUpdates, boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception { + void testClustering(boolean clusteringAsRow, boolean doUpdates, boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception { // set low compaction small File Size to generate more file groups. HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder() .forTable("test-trip-table") @@ -148,13 +158,22 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese assertEquals(allFiles.length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); // Do the clustering and validate - doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen); + doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen, clusteringAsRow); } } + private static Stream testClusteringWithNoBaseFiles() { + return Stream.of( + Arguments.of(true, true), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false) + ); + } + @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testClusteringWithNoBaseFiles(boolean doUpdates) throws Exception { + @MethodSource + void testClusteringWithNoBaseFiles(boolean clusteringAsRow, boolean doUpdates) throws Exception { // set low compaction small File Size to generate more file groups. HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder() .forTable("test-trip-table") @@ -217,7 +236,7 @@ void testClusteringWithNoBaseFiles(boolean doUpdates) throws Exception { assertEquals(dataGen.getPartitionPaths().length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); // do the clustering and validate - doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen); + doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen, clusteringAsRow); } } @@ -225,7 +244,13 @@ private void doClusteringAndValidate(SparkRDDWriteClient client, String clusteringCommitTime, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, - HoodieTestDataGenerator dataGen) { + HoodieTestDataGenerator dataGen, + boolean clusteringAsRow) { + if (clusteringAsRow) { + client.getConfig().setValue(HoodieClusteringConfig.CLUSTERING_AS_ROW, "true"); + client.getConfig().setAll(getPropertiesForKeyGen()); + } + client.cluster(clusteringCommitTime, true); metaClient = HoodieTableMetaClient.reload(metaClient); final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala index 17715627fef38..29991378665e8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala @@ -25,15 +25,16 @@ import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} import org.apache.hudi.testutils.HoodieClientTestBase -import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieFileIndex} +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions} import org.apache.spark.sql._ import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.Arguments.arguments import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import java.util +import java.util.{function, stream} import scala.collection.JavaConversions._ @Tag("functional") @@ -84,7 +85,8 @@ class TestLayoutOptimization extends HoodieClientTestBase { @ParameterizedTest @MethodSource(Array("testLayoutOptimizationParameters")) - def testLayoutOptimizationFunctional(tableType: String, + def testLayoutOptimizationFunctional(clusteringAsRow: String, + tableType: String, layoutOptimizationStrategy: String, spatialCurveCompositionStrategy: String): Unit = { val curveCompositionStrategy = @@ -112,6 +114,7 @@ class TestLayoutOptimization extends HoodieClientTestBase { .option("hoodie.clustering.plan.strategy.small.file.limit", "629145600") .option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString) .option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 * 1024 * 1024L)) + .option(HoodieClusteringConfig.CLUSTERING_AS_ROW.key(), clusteringAsRow) .option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_STRATEGY.key(), layoutOptimizationStrategy) .option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD.key(), curveCompositionStrategy) .option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat,begin_lon") @@ -165,17 +168,26 @@ class TestLayoutOptimization extends HoodieClientTestBase { object TestLayoutOptimization { def testLayoutOptimizationParameters(): java.util.stream.Stream[Arguments] = { java.util.stream.Stream.of( - arguments("COPY_ON_WRITE", "linear", null), - arguments("COPY_ON_WRITE", "z-order", "direct"), - arguments("COPY_ON_WRITE", "z-order", "sample"), - arguments("COPY_ON_WRITE", "hilbert", "direct"), - arguments("COPY_ON_WRITE", "hilbert", "sample"), - - arguments("MERGE_ON_READ", "linear", null), - arguments("MERGE_ON_READ", "z-order", "direct"), - arguments("MERGE_ON_READ", "z-order", "sample"), - arguments("MERGE_ON_READ", "hilbert", "direct"), - arguments("MERGE_ON_READ", "hilbert", "sample") - ) + Seq("COPY_ON_WRITE", "linear", null), + Seq("COPY_ON_WRITE", "z-order", "direct"), + Seq("COPY_ON_WRITE", "z-order", "sample"), + Seq("COPY_ON_WRITE", "hilbert", "direct"), + Seq("COPY_ON_WRITE", "hilbert", "sample"), + Seq("MERGE_ON_READ", "linear", null), + Seq("MERGE_ON_READ", "z-order", "direct"), + Seq("MERGE_ON_READ", "z-order", "sample"), + Seq("MERGE_ON_READ", "hilbert", "direct"), + Seq("MERGE_ON_READ", "hilbert", "sample") + ).flatMap(new function.Function[Seq[String], util.stream.Stream[Arguments]] { + override def apply(args: Seq[String]): stream.Stream[Arguments] = { + val enableRowClusteringArgs = "true" +: args + val disableRowClusteringArgs = "false" +: args + + java.util.stream.Stream.of( + Arguments.of(enableRowClusteringArgs.toArray:_*), + Arguments.of(disableRowClusteringArgs.toArray:_*) + ) + } + }) } } From 25f1afd8523f9979a5fa92f30cfe54c3d4574c45 Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 11 Jul 2022 16:00:01 +0800 Subject: [PATCH 03/18] Fix wrong config --- .../spark3/internal/HoodieDataSourceInternalBatchWrite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java index ed7072546389a..8ee63edc3dadf 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java @@ -73,7 +73,7 @@ public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - if (writeConfig.shouldAutoCommit()) { + if (writeConfig.bulkInsertRowAutoCommit()) { dataSourceInternalWriterHelper.createInflightCommit(); } @@ -100,7 +100,7 @@ public void commit(WriterCommitMessage[] messages) { List writeStatusList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) .flatMap(m -> m.getWriteStatuses().stream()) .collect(Collectors.toList()); - if (writeConfig.shouldAutoCommit()) { + if (writeConfig.bulkInsertRowAutoCommit()) { List writeStatList = writeStatusList.stream() .map(HoodieInternalWriteStatus::getStat).collect(Collectors.toList()); dataSourceInternalWriterHelper.commit(writeStatList); From fa7c1009a90e218285bf673ef4219d5f12b014e7 Mon Sep 17 00:00:00 2001 From: Rex An Date: Thu, 28 Jul 2022 18:59:57 +0800 Subject: [PATCH 04/18] Follow origin name style --- .../apache/hudi/io/storage/row/HoodieRowCreateHandle.java | 4 ++-- .../java/org/apache/hudi/common/model/HoodieRecord.java | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 56a718ef6da0b..90b69e54507c2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -160,8 +160,8 @@ private void writeRow(InternalRow row) { InternalRow updatedRow; if (preserveMetadata) { - updatedRow = new HoodieInternalRow(row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_POS), - row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_POS), + updatedRow = new HoodieInternalRow(row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD), + row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD), recordKey, partitionPath, fileName, row, true); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index 6d6ae7ffdbbba..65d27aeb5b1b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -60,10 +60,8 @@ public abstract class HoodieRecord implements Serializable { public static int RECORD_KEY_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD); public static int PARTITION_PATH_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD); public static int FILENAME_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD); - - public static int COMMIT_TIME_METADATA_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_TIME_METADATA_FIELD); - - public static int COMMIT_SEQNO_METADATA_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_SEQNO_METADATA_FIELD); + public static int COMMIT_TIME_METADATA_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_TIME_METADATA_FIELD); + public static int COMMIT_SEQNO_METADATA_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(COMMIT_SEQNO_METADATA_FIELD); /** * Identifies the record across the table. From 1c913457d2dd531fd1ecae6b0d60e600f59e261b Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 1 Aug 2022 15:10:36 +0800 Subject: [PATCH 05/18] Rebase codes --- .../apache/hudi/config/HoodieWriteConfig.java | 4 ++++ .../MultipleSparkJobExecutionStrategy.java | 7 +++++- .../io/storage/row/HoodieRowCreateHandle.java | 2 +- .../hudi/HoodieDatasetBulkInsertHelper.java | 0 .../hudi/HoodieDatasetBulkInsertHelper.scala | 24 ++++++++++++------- 5 files changed, 27 insertions(+), 10 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 63d2a29431e2b..b2cc3f66e98a9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1165,6 +1165,10 @@ public boolean populateMetaFields() { return getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); } + public boolean bulkInsertPreserverMetadata() { + return getBooleanOrDefault(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA); + } + /** * compaction properties. */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index b16bf4c039535..855eedca46ebf 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -198,7 +198,12 @@ protected Map buildHoodieRowParameters(int numOutputGroups, Stri params.compute(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), (k, v) -> writeConfig.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)); params.put("hoodie.datasource.write.operation", "bulk_insert"); params.put("hoodie.instant.time", instantTime); - params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), String.valueOf(preserveHoodieMetadata)); + if (!writeConfig.populateMetaFields() && preserveHoodieMetadata) { + LOG.warn("Will setting preserveHoodieMetadata to false as populateMetaFields is false"); + params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), "false"); + } else { + params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), String.valueOf(preserveHoodieMetadata)); + } configRowPartitioner(strategyParams, params); return params; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 90b69e54507c2..8b018a5d77d0f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -106,7 +106,7 @@ public HoodieRowCreateHandle(HoodieTable table, this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(), writeConfig.getWriteStatusFailureFraction()); - this.preserveMetadata = writeConfig.getBoolean(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA); + this.preserveMetadata = writeConfig.bulkInsertPreserverMetadata(); writeStatus.setPartitionPath(partitionPath); writeStatus.setFileId(fileId); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 09f1fac2c874d..c24c6dc9bd0a6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{StringType, StructField, StructType} -import org.apache.spark.sql.{DataFrame, Dataset, HoodieUnsafeUtils, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, HoodieUnsafeUtils, Row} import org.apache.spark.unsafe.types.UTF8String import scala.collection.JavaConverters.asScalaBufferConverter @@ -53,13 +53,17 @@ object HoodieDatasetBulkInsertHelper extends Logging { partitioner: BulkInsertPartitioner[Dataset[Row]], shouldDropPartitionColumns: Boolean): Dataset[Row] = { val populateMetaFields = config.populateMetaFields() - val schema = df.schema + val preserveMetadata = config.bulkInsertPreserverMetadata() - val keyGeneratorClassName = config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME, - "Key-generator class name is required") + val prependedRdd = if (preserveMetadata) { + df.queryExecution.toRdd + } else { + val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) + val schema = dfWithoutMetaCols.schema - val prependedRdd: RDD[InternalRow] = - df.queryExecution.toRdd.mapPartitions { iter => + val keyGeneratorClassName = config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME, + "Key-generator class name is required") + dfWithoutMetaCols.queryExecution.toRdd.mapPartitions { iter => val keyGenerator = ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps)) .asInstanceOf[SparkKeyGeneratorInterface] @@ -78,7 +82,8 @@ object HoodieDatasetBulkInsertHelper extends Logging { // TODO use mutable row, avoid re-allocating new HoodieInternalRow(commitTimestamp, commitSeqNo, recordKey, partitionPath, filename, row, false) } - } + }.asInstanceOf[RDD[InternalRow]] + } val metaFields = Seq( StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, StringType), @@ -87,7 +92,10 @@ object HoodieDatasetBulkInsertHelper extends Logging { StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, StringType), StructField(HoodieRecord.FILENAME_METADATA_FIELD, StringType)) - val updatedSchema = StructType(metaFields ++ schema.fields) + val originalFieldsWithoutHoodieMeta = df.schema.fields + .filter(f => !HoodieRecord.HOODIE_META_COLUMNS.contains(f.name)) + + val updatedSchema = StructType(metaFields ++ originalFieldsWithoutHoodieMeta) val updatedDF = if (populateMetaFields && config.shouldCombineBeforeInsert) { val dedupedRdd = dedupeRows(prependedRdd, updatedSchema, config.getPreCombineField, SparkHoodieIndexFactory.isGlobalIndex(config)) From f9bed4815c7869ae125dd0ecdc8156886e1b671d Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 5 Sep 2022 10:39:02 +0800 Subject: [PATCH 06/18] Address review comments --- .../hudi/config/HoodieClusteringConfig.java | 8 -- .../apache/hudi/config/HoodieWriteConfig.java | 20 ----- .../MultipleSparkJobExecutionStrategy.java | 53 ++++++------- .../SparkSingleFileSortExecutionStrategy.java | 21 +++-- .../SparkSortAndSizeExecutionStrategy.java | 20 +++-- .../RowCustomColumnsSortPartitioner.java | 4 - .../RowSpatialCurveSortPartitioner.java | 7 +- .../io/storage/row/HoodieRowCreateHandle.java | 20 ++++- .../BulkInsertDataInternalWriterHelper.java | 16 +++- .../commit/SparkBulkInsertRowWriter.java | 76 +++++++++++++++++++ .../hudi/common/config/HoodieConfig.java | 4 + .../DataSourceInternalWriterHelper.java | 5 +- .../scala/org/apache/hudi/DefaultSource.scala | 10 +-- .../hudi/HoodieDatasetBulkInsertHelper.scala | 24 ++---- .../apache/hudi/HoodieSparkSqlWriter.scala | 33 +++----- .../HoodieBulkInsertDataInternalWriter.java | 1 + .../HoodieDataSourceInternalWriter.java | 26 +------ .../HoodieBulkInsertDataInternalWriter.java | 2 +- .../HoodieDataSourceInternalBatchWrite.java | 27 +------ 19 files changed, 198 insertions(+), 179 deletions(-) rename {hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal => hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit}/BulkInsertDataInternalWriterHelper.java (90%) create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index d1a8fc03f7755..d74e462b8da9e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -137,14 +137,6 @@ public class HoodieClusteringConfig extends HoodieConfig { .defaultValue("4") .sinceVersion("0.9.0") .withDocumentation("Config to control frequency of async clustering"); - - public static final ConfigProperty CLUSTERING_AS_ROW = ConfigProperty - .key("hoodie.clustering.as.row") - .defaultValue(false) - .sinceVersion("0.12.0") - .withDocumentation("If set to true, will perform bulk_insert operation directly using the spark native " - + "`Row` representation, avoiding any additional conversion costs."); - public static final ConfigProperty PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions") .defaultValue("0") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index b2cc3f66e98a9..cddb58cb0731a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -216,18 +216,6 @@ public class HoodieWriteConfig extends HoodieConfig { .noDefaultValue() .withDocumentation("The unique id for each write operation, HoodieInternalWriteStatusCoordinator will use " + "this id to identify the related write statuses"); - public static final ConfigProperty BULKINSERT_PRESERVE_METADATA = ConfigProperty - .key("hoodie.datasource.preserve.metadata") - .defaultValue(false) - .withDocumentation("Whether to preserve metadata from the read source, this is only take effective " - + "When using bulk_insert and row writer is enabled"); - - public static final ConfigProperty BULKINSERT_ROW_AUTO_COMMIT = ConfigProperty - .key("hoodie.bulkinsert.row.auto.commit") - .defaultValue(true) - .withDocumentation("Whether to create request, inflight and post commit automatically, this can be turned " - + "off to perform inspection of the uncommitted write before deciding to commit."); - public static final ConfigProperty UPSERT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.upsert.shuffle.parallelism") .defaultValue("200") @@ -999,10 +987,6 @@ public Boolean shouldAutoCommit() { return getBoolean(AUTO_COMMIT_ENABLE); } - public Boolean bulkInsertRowAutoCommit() { - return getBoolean(BULKINSERT_ROW_AUTO_COMMIT); - } - public Boolean shouldAssumeDatePartitioning() { return metadataConfig.shouldAssumeDatePartitioning(); } @@ -1165,10 +1149,6 @@ public boolean populateMetaFields() { return getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); } - public boolean bulkInsertPreserverMetadata() { - return getBooleanOrDefault(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA); - } - /** * compaction properties. */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 855eedca46ebf..d9b01a10e56dd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -38,6 +38,7 @@ import org.apache.hudi.common.model.RewriteAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FutureUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -48,6 +49,7 @@ import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory; +import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.execution.bulkinsert.RDDSpatialCurveSortPartitioner; import org.apache.hudi.execution.bulkinsert.RowCustomColumnsSortPartitioner; @@ -57,7 +59,6 @@ import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenUtils; -import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; @@ -81,7 +82,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -111,7 +111,7 @@ public HoodieWriteMetadata> performClustering(final Hood Stream> writeStatusesStream = FutureUtils.allOf( clusteringPlan.getInputGroups().stream() .map(inputGroup -> { - if (Boolean.parseBoolean(getWriteConfig().getString(HoodieClusteringConfig.CLUSTERING_AS_ROW))) { + if (getWriteConfig().getBooleanOrDefault("hoodie.datasource.write.row.writer.enable", false)) { return runClusteringForGroupAsyncWithRow(inputGroup, clusteringPlan.getStrategy().getStrategyParams(), Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), @@ -187,25 +187,14 @@ protected HoodieData performRowWrite(Dataset inputRecords, Map } } - protected Map buildHoodieRowParameters(int numOutputGroups, String instantTime, Map strategyParams, boolean preserveHoodieMetadata) { - HashMap params = new HashMap<>(); - HoodieWriteConfig writeConfig = getWriteConfig(); - params.put(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key(), String.valueOf(numOutputGroups)); - params.put(HoodieWriteConfig.BULKINSERT_ROW_AUTO_COMMIT.key(), String.valueOf(false)); - params.put(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED.key(), String.valueOf(true)); - params.compute(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), (k, v) -> writeConfig.getKeyGeneratorClass()); - params.compute(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), (k, v) -> writeConfig.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); - params.compute(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), (k, v) -> writeConfig.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)); - params.put("hoodie.datasource.write.operation", "bulk_insert"); - params.put("hoodie.instant.time", instantTime); - if (!writeConfig.populateMetaFields() && preserveHoodieMetadata) { - LOG.warn("Will setting preserveHoodieMetadata to false as populateMetaFields is false"); - params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), "false"); - } else { - params.put(HoodieWriteConfig.BULKINSERT_PRESERVE_METADATA.key(), String.valueOf(preserveHoodieMetadata)); - } - configRowPartitioner(strategyParams, params); - return params; + protected BulkInsertPartitioner> getRowPartitioner(Map strategyParams, + Schema schema) { + return getPartitioner(strategyParams, schema, true); + } + + protected BulkInsertPartitioner>> getRDDPartitioner(Map strategyParams, + Schema schema) { + return getPartitioner(strategyParams, schema, false); } /** @@ -213,9 +202,10 @@ protected Map buildHoodieRowParameters(int numOutputGroups, Stri * * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. * @param schema Schema of the data including metadata fields. - * @return {@link RDDCustomColumnsSortPartitioner} if sort columns are provided, otherwise empty. */ - protected BulkInsertPartitioner>> getPartitioner(Map strategyParams, Schema schema) { + protected BulkInsertPartitioner getPartitioner(Map strategyParams, + Schema schema, + boolean isRowPartitioner) { Option orderByColumnsOpt = Option.ofNullable(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key())) .map(listStr -> listStr.split(",")); @@ -225,19 +215,24 @@ protected BulkInsertPartitioner>> getPartitioner(Map readRecordsForGroupAsRow(JavaSparkContext jsc, if (hasLogFiles) { String compactionFractor = Option.ofNullable(getWriteConfig().getString("compaction.memory.fraction")) .orElse("0.75"); - String[] paths = new String[baseFilePaths.length + deltaPaths.length]; - System.arraycopy(baseFilePaths, 0, paths, 0, baseFilePaths.length); - System.arraycopy(deltaPaths, 0, paths, baseFilePaths.length, deltaPaths.length); + String[] paths = CollectionUtils.combine(baseFilePaths, deltaPaths); inputRecords = sqlContext.read() .format("org.apache.hudi") .option("hoodie.datasource.query.type", "snapshot") diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index c95d5398f2696..20145b343635c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -33,12 +33,12 @@ import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; import org.apache.avro.Schema; +import org.apache.hudi.table.action.commit.SparkBulkInsertRowWriter; 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.HashMap; import java.util.List; import java.util.Map; @@ -70,11 +70,20 @@ public HoodieData performClusteringWithRecordsRow(Dataset inpu } LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); - HashMap params = new HashMap<>( - buildHoodieRowParameters(numOutputGroups, instantTime, strategyParams, preserveHoodieMetadata)); - params.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(Long.MAX_VALUE)); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() + .withBulkInsertParallelism(numOutputGroups) + .withProps(getWriteConfig().getProps()).build(); + + boolean shouldPreserveHoodieMetadata = preserveHoodieMetadata; + if (!newConfig.populateMetaFields() && preserveHoodieMetadata) { + LOG.warn("Will setting preserveHoodieMetadata to false as populateMetaFields is false"); + shouldPreserveHoodieMetadata = false; + } + + newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE)); - return performRowWrite(inputRecords, params); + return SparkBulkInsertRowWriter.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata); } @Override @@ -97,6 +106,6 @@ public HoodieData performClusteringWithRecordsRDD(HoodieData) 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(), preserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 7f5120c9d8e42..33ad16f909c14 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -31,12 +31,12 @@ import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; import org.apache.avro.Schema; +import org.apache.hudi.table.action.commit.SparkBulkInsertRowWriter; 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.HashMap; import java.util.List; import java.util.Map; @@ -60,12 +60,20 @@ public HoodieData performClusteringWithRecordsRow(Dataset inpu String instantTime, Map strategyParams, Schema schema, List fileGroupIdList, boolean preserveHoodieMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() + .withBulkInsertParallelism(numOutputGroups) + .withProps(getWriteConfig().getProps()).build(); - HashMap params = new HashMap<>( - buildHoodieRowParameters(numOutputGroups, instantTime, strategyParams, preserveHoodieMetadata)); - params.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); + boolean shouldPreserveHoodieMetadata = preserveHoodieMetadata; + if (!newConfig.populateMetaFields() && preserveHoodieMetadata) { + LOG.warn("Will setting preserveHoodieMetadata to false as populateMetaFields is false"); + shouldPreserveHoodieMetadata = false; + } + + newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); - return performRowWrite(inputRecords, params); + return SparkBulkInsertRowWriter.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata); } @Override @@ -79,6 +87,6 @@ public HoodieData performClusteringWithRecordsRDD(final HoodieData< .withProps(getWriteConfig().getProps()).build(); newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); return (HoodieData) SparkBulkInsertHelper.newInstance() - .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); + .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getRDDPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java index 324c462cbb947..ceeb2b3fe8f00 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowCustomColumnsSortPartitioner.java @@ -53,10 +53,6 @@ public boolean arePartitionRecordsSorted() { return true; } - public String[] getSortColumnNames() { - return sortColumnNames; - } - private String[] getSortColumnName(HoodieWriteConfig config) { return Arrays.stream(config.getUserDefinedBulkInsertPartitionerSortColumns().split(",")) .map(String::trim).toArray(String[]::new); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java index b3f69c3ae6f86..7f9a9ac07da6a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java @@ -21,26 +21,27 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.sort.SpaceCurveSortingHelper; +import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import java.util.Arrays; import java.util.List; -public class RowSpatialCurveSortPartitioner extends RowCustomColumnsSortPartitioner { +public class RowSpatialCurveSortPartitioner implements BulkInsertPartitioner> { private final String[] orderByColumns; private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy; private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType; public RowSpatialCurveSortPartitioner(HoodieWriteConfig config) { - super(config); this.layoutOptStrategy = config.getLayoutOptimizationStrategy(); if (config.getClusteringSortColumns() != null) { this.orderByColumns = Arrays.stream(config.getClusteringSortColumns().split(",")) .map(String::trim).toArray(String[]::new); } else { - this.orderByColumns = getSortColumnNames(); + throw new IllegalArgumentException("The config " + + HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key() + " must be provided"); } this.curveCompositionStrategyType = config.getLayoutOptimizationCurveBuildMethod(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 8b018a5d77d0f..648529575b811 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -70,7 +70,7 @@ public class HoodieRowCreateHandle implements Serializable { private final UTF8String commitTime; private final Function seqIdGenerator; - private final boolean preserveMetadata; + private final boolean preserveHoodieMetadata; private final HoodieTimer currTimer; @@ -86,6 +86,20 @@ public HoodieRowCreateHandle(HoodieTable table, long taskId, long taskEpochId, StructType structType) { + this(table, writeConfig, partitionPath, fileId, instantTime, taskPartitionId, taskId, taskEpochId, + structType, false); + } + + public HoodieRowCreateHandle(HoodieTable table, + HoodieWriteConfig writeConfig, + String partitionPath, + String fileId, + String instantTime, + int taskPartitionId, + long taskId, + long taskEpochId, + StructType structType, + boolean preserveHoodieMetadata) { this.partitionPath = partitionPath; this.table = table; this.writeConfig = writeConfig; @@ -106,7 +120,7 @@ public HoodieRowCreateHandle(HoodieTable table, this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(), writeConfig.getWriteStatusFailureFraction()); - this.preserveMetadata = writeConfig.bulkInsertPreserverMetadata(); + this.preserveHoodieMetadata = preserveHoodieMetadata; writeStatus.setPartitionPath(partitionPath); writeStatus.setFileId(fileId); @@ -159,7 +173,7 @@ private void writeRow(InternalRow row) { UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); InternalRow updatedRow; - if (preserveMetadata) { + if (preserveHoodieMetadata) { updatedRow = new HoodieInternalRow(row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD), row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD), recordKey, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java similarity index 90% rename from hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java index f7918cf3fd9f1..c15c4e647b6a9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.hudi.internal; +package org.apache.hudi.table.action.commit; -import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; @@ -66,6 +65,7 @@ public class BulkInsertDataInternalWriterHelper { private final String fileIdPrefix; private final Map handles = new HashMap<>(); private final boolean populateMetaFields; + private final boolean preserveHoodieMetadata; private final Option keyGeneratorOpt; private final boolean simpleKeyGen; private final int simplePartitionFieldIndex; @@ -81,6 +81,13 @@ public class BulkInsertDataInternalWriterHelper { public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, boolean populateMetaFields, boolean arePartitionRecordsSorted) { + this(hoodieTable, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, + populateMetaFields, arePartitionRecordsSorted, false); + } + + public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, + boolean populateMetaFields, boolean arePartitionRecordsSorted, boolean preserveHoodieMetadata) { this.hoodieTable = hoodieTable; this.writeConfig = writeConfig; this.instantTime = instantTime; @@ -89,6 +96,7 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo this.taskEpochId = taskEpochId; this.structType = structType; this.populateMetaFields = populateMetaFields; + this.preserveHoodieMetadata = preserveHoodieMetadata; this.arePartitionRecordsSorted = arePartitionRecordsSorted; this.fileIdPrefix = UUID.randomUUID().toString(); @@ -118,7 +126,7 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo private Option getKeyGenerator(Properties properties) { TypedProperties typedProperties = new TypedProperties(); typedProperties.putAll(properties); - if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key()).equals(NonpartitionedKeyGenerator.class.getName())) { + if (properties.get(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key()).equals(NonpartitionedKeyGenerator.class.getName())) { return Option.empty(); // Do not instantiate NonPartitionKeyGen } else { try { @@ -199,7 +207,7 @@ private HoodieRowCreateHandle getRowCreateHandle(String partitionPath) throws IO private HoodieRowCreateHandle createHandle(String partitionPath) { return new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(), - instantTime, taskPartitionId, taskId, taskEpochId, structType); + instantTime, taskPartitionId, taskId, taskEpochId, structType, preserveHoodieMetadata); } private String getNextFileId() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java new file mode 100644 index 0000000000000..a1a9d49811e37 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java @@ -0,0 +1,76 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; + +import java.util.Iterator; +import java.util.List; + +public class SparkBulkInsertRowWriter { + + public static HoodieData bulkInsert(Dataset dataset, + String instantTime, + HoodieTable table, + HoodieWriteConfig writeConfig, + BulkInsertPartitioner> partitioner, + int parallelism, + boolean preserveHoodieMetadata) { + Dataset repartitionedDataset = partitioner.repartitionRecords(dataset, parallelism); + + boolean arePartitionRecordsSorted = partitioner.arePartitionRecordsSorted(); + StructType schema = dataset.schema(); + List writeStatuses = repartitionedDataset.queryExecution().toRdd().toJavaRDD().mapPartitions( + (FlatMapFunction, WriteStatus>) rowIterator -> { + TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier(); + int taskPartitionId = taskContextSupplier.getPartitionIdSupplier().get(); + long taskId = taskContextSupplier.getStageIdSupplier().get(); + long taskEpochId = taskContextSupplier.getAttemptIdSupplier().get(); + + final BulkInsertDataInternalWriterHelper writer = + new BulkInsertDataInternalWriterHelper(table, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, + schema, writeConfig.populateMetaFields(), arePartitionRecordsSorted, preserveHoodieMetadata); + while (rowIterator.hasNext()) { + writer.write(rowIterator.next()); + } + return writer.getWriteStatuses() + .stream() + .map(internalWriteStatus -> { + WriteStatus status = new WriteStatus( + internalWriteStatus.isTrackSuccessRecords(), internalWriteStatus.getFailureFraction()); + status.setFileId(internalWriteStatus.getFileId()); + status.setTotalRecords(internalWriteStatus.getTotalRecords()); + status.setPartitionPath(internalWriteStatus.getPartitionPath()); + status.setStat(internalWriteStatus.getStat()); + return status; + }).iterator(); + }).collect(); + return table.getContext().parallelize(writeStatuses); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index 6d4236b048d2d..366d19fe6ebc0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -166,6 +166,10 @@ public Boolean getBoolean(ConfigProperty configProperty) { return rawValue.map(v -> Boolean.parseBoolean(v.toString())).orElse(null); } + public boolean getBooleanOrDefault(String key, boolean defaultVal) { + return Option.ofNullable(props.getProperty(key)).map(Boolean::parseBoolean).orElse(defaultVal); + } + public boolean getBooleanOrDefault(ConfigProperty configProperty) { Option rawValue = getRawValue(configProperty); return rawValue.map(v -> Boolean.parseBoolean(v.toString())) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index 0de0cf7c7c575..3a349473b2201 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -63,6 +63,7 @@ public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writ this.extraMetadata = extraMetadata; this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig); writeClient.setOperationType(operationType); + writeClient.startCommitWithTime(instantTime); this.metaClient = HoodieTableMetaClient.builder().setConf(configuration).setBasePath(writeConfig.getBasePath()).build(); this.metaClient.validateTableProperties(writeConfig.getProps()); @@ -95,10 +96,6 @@ public void abort() { writeClient.close(); } - public void createRequestedCommit() { - writeClient.startCommitWithTime(instantTime); - } - public void createInflightCommit() { metaClient.getActiveTimeline().transitionRequestedToInflight( new HoodieInstant(State.REQUESTED, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 38b418efd857f..d5cbf020ed3e5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -175,16 +175,14 @@ class DefaultSource extends RelationProvider mode: SaveMode, optParams: Map[String, String], df: DataFrame): BaseRelation = { - val colsWithoutHoodieMeta = df.schema - .filter(field => !HoodieRecord.HOODIE_META_COLUMNS.contains(field.name)) - .foldLeft(new StructType())((s, f) => s.add(f)) + val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) if (optParams.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)) { - HoodieSparkSqlWriter.bootstrap(sqlContext, mode, optParams, df) + HoodieSparkSqlWriter.bootstrap(sqlContext, mode, optParams, dfWithoutMetaCols) } else { - HoodieSparkSqlWriter.write(sqlContext, mode, optParams, df) + HoodieSparkSqlWriter.write(sqlContext, mode, optParams, dfWithoutMetaCols) } - new HoodieEmptyRelation(sqlContext, colsWithoutHoodieMeta) + new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema) } override def createSink(sqlContext: SQLContext, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index c24c6dc9bd0a6..09f1fac2c874d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{StringType, StructField, StructType} -import org.apache.spark.sql.{Column, DataFrame, Dataset, HoodieUnsafeUtils, Row} +import org.apache.spark.sql.{DataFrame, Dataset, HoodieUnsafeUtils, Row} import org.apache.spark.unsafe.types.UTF8String import scala.collection.JavaConverters.asScalaBufferConverter @@ -53,17 +53,13 @@ object HoodieDatasetBulkInsertHelper extends Logging { partitioner: BulkInsertPartitioner[Dataset[Row]], shouldDropPartitionColumns: Boolean): Dataset[Row] = { val populateMetaFields = config.populateMetaFields() - val preserveMetadata = config.bulkInsertPreserverMetadata() + val schema = df.schema - val prependedRdd = if (preserveMetadata) { - df.queryExecution.toRdd - } else { - val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) - val schema = dfWithoutMetaCols.schema + val keyGeneratorClassName = config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME, + "Key-generator class name is required") - val keyGeneratorClassName = config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME, - "Key-generator class name is required") - dfWithoutMetaCols.queryExecution.toRdd.mapPartitions { iter => + val prependedRdd: RDD[InternalRow] = + df.queryExecution.toRdd.mapPartitions { iter => val keyGenerator = ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps)) .asInstanceOf[SparkKeyGeneratorInterface] @@ -82,8 +78,7 @@ object HoodieDatasetBulkInsertHelper extends Logging { // TODO use mutable row, avoid re-allocating new HoodieInternalRow(commitTimestamp, commitSeqNo, recordKey, partitionPath, filename, row, false) } - }.asInstanceOf[RDD[InternalRow]] - } + } val metaFields = Seq( StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, StringType), @@ -92,10 +87,7 @@ object HoodieDatasetBulkInsertHelper extends Logging { StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, StringType), StructField(HoodieRecord.FILENAME_METADATA_FIELD, StringType)) - val originalFieldsWithoutHoodieMeta = df.schema.fields - .filter(f => !HoodieRecord.HOODIE_META_COLUMNS.contains(f.name)) - - val updatedSchema = StructType(metaFields ++ originalFieldsWithoutHoodieMeta) + val updatedSchema = StructType(metaFields ++ schema.fields) val updatedDF = if (populateMetaFields && config.shouldCombineBeforeInsert) { val dedupedRdd = dedupeRows(prependedRdd, updatedSchema, config.getPreCombineField, SparkHoodieIndexFactory.isGlobalIndex(config)) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 6f75c9ac9d1a3..8780519ea7971 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -124,7 +124,7 @@ object HoodieSparkSqlWriter { jsc.setLocalProperty("spark.scheduler.pool", SparkConfigs.SPARK_DATASOURCE_WRITER_POOL_NAME) } } - + val instantTime = HoodieActiveTimeline.createNewInstantTime() val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(hoodieConfig.getProps)) if (mode == SaveMode.Ignore && tableExists) { @@ -170,11 +170,6 @@ object HoodieSparkSqlWriter { val commitActionType = CommitUtils.getCommitActionType(operation, tableConfig.getTableType) val dropPartitionColumns = hoodieConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS) - val instantTime = Option(hoodieConfig.getString(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY)) match { - case Some(instant) => instant - case _ => HoodieActiveTimeline.createNewInstantTime() - } - // short-circuit if bulk_insert via row is enabled. // scalastyle:off if (hoodieConfig.getBoolean(ENABLE_ROW_WRITER) && @@ -185,14 +180,11 @@ object HoodieSparkSqlWriter { } // scalastyle:on - val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS:_*) - val reconcileSchema = parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) = operation match { case WriteOperationType.DELETE => { - val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, dfWithoutMetaCols, reconcileSchema) - // Convert to RDD[HoodieKey] + val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) // Convert to RDD[HoodieKey] val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD() if (!tableExists) { @@ -228,7 +220,7 @@ object HoodieSparkSqlWriter { val partitionColsToDelete = parameters(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key()).split(",") java.util.Arrays.asList(partitionColsToDelete: _*) } else { - val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, dfWithoutMetaCols, reconcileSchema) + val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) genericRecords.map(gr => keyGenerator.getKey(gr).getPartitionPath).toJavaRDD().distinct().collect() } // Create a HoodieWriteClient & issue the delete. @@ -249,7 +241,7 @@ object HoodieSparkSqlWriter { classOf[org.apache.avro.Schema])) // TODO(HUDI-4472) revisit and simplify schema handling - val sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dfWithoutMetaCols.schema, structName, nameSpace) + val sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) val latestTableSchema = getLatestTableSchema(fs, basePath, sparkContext).getOrElse(sourceSchema) val schemaEvolutionEnabled = parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean @@ -340,7 +332,7 @@ object HoodieSparkSqlWriter { // Check for errors and commit the write. val (writeSuccessful, compactionInstant, clusteringInstant) = - commitAndPerformPostOperations(sqlContext.sparkSession, dfWithoutMetaCols.schema, + commitAndPerformPostOperations(sqlContext.sparkSession, df.schema, writeResult, parameters, writeClient, tableConfig, jsc, TableInstantInfo(basePath, instantTime, commitActionType, operation)) @@ -434,7 +426,6 @@ object HoodieSparkSqlWriter { df: DataFrame, hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty, hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty): Boolean = { - val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS:_*) assert(optParams.get("path").exists(!StringUtils.isNullOrEmpty(_)), "'path' must be set") val path = optParams("path") @@ -454,9 +445,9 @@ object HoodieSparkSqlWriter { val bootstrapIndexClass = hoodieConfig.getStringOrDefault(INDEX_CLASS_NAME) var schema: String = null - if (dfWithoutMetaCols.schema.nonEmpty) { + if (df.schema.nonEmpty) { val (structName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName) - schema = AvroConversionUtils.convertStructTypeToAvroSchema(dfWithoutMetaCols.schema, structName, namespace).toString + schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, namespace).toString } else { schema = HoodieAvroUtils.getNullSchema.toString } @@ -514,7 +505,7 @@ object HoodieSparkSqlWriter { } finally { writeClient.close() } - val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, dfWithoutMetaCols.schema) + val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema) metaSyncSuccess } } @@ -541,15 +532,11 @@ object HoodieSparkSqlWriter { val dropPartitionColumns = parameters.get(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key()).map(_.toBoolean) .getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()) // register classes & schemas - val colsWithoutHoodieMeta = df.schema - .filter(field => !HoodieRecord.HOODIE_META_COLUMNS.contains(field.name)) - .foldLeft(new StructType())((s, f) => s.add(f)) - val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName) sparkContext.getConf.registerKryoClasses( Array(classOf[org.apache.avro.generic.GenericData], classOf[org.apache.avro.Schema])) - var schema = AvroConversionUtils.convertStructTypeToAvroSchema(colsWithoutHoodieMeta, structName, nameSpace) + var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) if (dropPartitionColumns) { schema = generateSchemaWithoutPartitionColumns(partitionColumns, schema) } @@ -601,7 +588,7 @@ object HoodieSparkSqlWriter { throw new HoodieException("Bulk insert using row writer is not supported with current Spark version." + " To use row writer please switch to spark 2 or spark 3") } - val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, colsWithoutHoodieMeta) + val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, df.schema) (syncHiveSuccess, common.util.Option.ofNullable(instantTime)) } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java index 8a8a50502913f..e57695c03bf3c 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java @@ -21,6 +21,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java index a7e26d4bc77fa..c4b21483e8fee 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -20,10 +20,8 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.client.HoodieInternalWriteStatusCoordinator; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hadoop.conf.Configuration; @@ -66,17 +64,11 @@ public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writ this.extraMetadataMap = DataSourceUtils.getExtraMetadata(dataSourceOptions.asMap()); this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, sparkSession, configuration, extraMetadataMap); - - if (writeConfig.bulkInsertRowAutoCommit()) { - this.dataSourceInternalWriterHelper.createRequestedCommit(); - } } @Override public DataWriterFactory createWriterFactory() { - if (writeConfig.bulkInsertRowAutoCommit()) { - dataSourceInternalWriterHelper.createInflightCommit(); - } + dataSourceInternalWriterHelper.createInflightCommit(); if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted); @@ -97,19 +89,9 @@ public void onDataWriterCommit(WriterCommitMessage message) { @Override public void commit(WriterCommitMessage[] messages) { - List writeStatusList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) - .flatMap(m -> m.getWriteStatuses().stream()) - .collect(Collectors.toList()); - if (writeConfig.bulkInsertRowAutoCommit()) { - List writeStatList = writeStatusList.stream() - .map(HoodieInternalWriteStatus::getStat).collect(Collectors.toList()); - dataSourceInternalWriterHelper.commit(writeStatList); - } - - Option.ofNullable(writeConfig.getString(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key())).map(id -> { - HoodieInternalWriteStatusCoordinator.get().assignStatuses(id, writeStatusList); - return true; - }); + List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); } @Override diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java index e9163c8244c2e..dc7f69e80a043 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java @@ -19,9 +19,9 @@ package org.apache.hudi.spark3.internal; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.internal.BulkInsertDataInternalWriterHelper; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; import org.apache.spark.sql.connector.write.WriterCommitMessage; diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java index 8ee63edc3dadf..fb5f609d79ef2 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java @@ -20,10 +20,8 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.client.HoodieInternalWriteStatusCoordinator; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.internal.DataSourceInternalWriterHelper; @@ -65,18 +63,11 @@ public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig this.extraMetadata = DataSourceUtils.getExtraMetadata(properties); this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, jss, hadoopConfiguration, extraMetadata); - - if (writeConfig.bulkInsertRowAutoCommit()) { - this.dataSourceInternalWriterHelper.createRequestedCommit(); - } } @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - if (writeConfig.bulkInsertRowAutoCommit()) { - dataSourceInternalWriterHelper.createInflightCommit(); - } - + dataSourceInternalWriterHelper.createInflightCommit(); if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted); @@ -97,19 +88,9 @@ public void onDataWriterCommit(WriterCommitMessage message) { @Override public void commit(WriterCommitMessage[] messages) { - List writeStatusList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) - .flatMap(m -> m.getWriteStatuses().stream()) - .collect(Collectors.toList()); - if (writeConfig.bulkInsertRowAutoCommit()) { - List writeStatList = writeStatusList.stream() - .map(HoodieInternalWriteStatus::getStat).collect(Collectors.toList()); - dataSourceInternalWriterHelper.commit(writeStatList); - } - - Option.ofNullable(writeConfig.getString(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key())).map(id -> { - HoodieInternalWriteStatusCoordinator.get().assignStatuses(id, writeStatusList); - return true; - }); + List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); } @Override From 687ec04dba4b1a2b82d1e7d7d48d0ef9c4be2b43 Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 5 Sep 2022 10:59:30 +0800 Subject: [PATCH 07/18] Remove unused classes --- .../HoodieInternalWriteStatusCoordinator.java | 55 ---------------- .../hudi/config/HoodieClusteringConfig.java | 1 + .../apache/hudi/config/HoodieWriteConfig.java | 5 -- .../MultipleSparkJobExecutionStrategy.java | 65 +------------------ .../apache/hudi/HoodieSparkSqlWriter.scala | 3 +- .../HoodieBulkInsertDataInternalWriter.java | 2 +- .../HoodieBulkInsertDataInternalWriter.java | 2 +- 7 files changed, 6 insertions(+), 127 deletions(-) delete mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java deleted file mode 100644 index 1111454de04e9..0000000000000 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatusCoordinator.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.client; - -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; - -public class HoodieInternalWriteStatusCoordinator { - private static final HoodieInternalWriteStatusCoordinator INSTANCE = new HoodieInternalWriteStatusCoordinator(); - private final ConcurrentHashMap> statusesMap; - private HoodieInternalWriteStatusCoordinator() { - this.statusesMap = new ConcurrentHashMap<>(); - } - - public static HoodieInternalWriteStatusCoordinator get() { - return INSTANCE; - } - - /** - * Call this to save writeStatuses corresponding with an identifyId. - */ - public void assignStatuses(String identifyId, List writeStatuses) { - statusesMap.put(identifyId, writeStatuses); - } - - /** - * Return related internalWriteStatuses by the identifyId. - */ - public List getWriteStatuses(String identifyId) { - return statusesMap.get(identifyId); - } - - /** - * Remove related internalWriteStatuses by the identifyId. - */ - public void removeStatuses(String identifyId) { - statusesMap.remove(identifyId); - } -} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index d74e462b8da9e..1180845a6ed8a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -137,6 +137,7 @@ public class HoodieClusteringConfig extends HoodieConfig { .defaultValue("4") .sinceVersion("0.9.0") .withDocumentation("Config to control frequency of async clustering"); + public static final ConfigProperty PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions") .defaultValue("0") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index cddb58cb0731a..61b12abb27370 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -211,11 +211,6 @@ public class HoodieWriteConfig extends HoodieConfig { + " optimally for common query patterns. For now we support a build-in user defined bulkinsert partitioner org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner" + " which can does sorting based on specified column values set by " + BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS.key()); - public static final ConfigProperty BULKINSERT_ROW_IDENTIFY_ID = ConfigProperty - .key("hoodie.bulkinsert.row.writestatus.id") - .noDefaultValue() - .withDocumentation("The unique id for each write operation, HoodieInternalWriteStatusCoordinator will use " - + "this id to identify the related write statuses"); public static final ConfigProperty UPSERT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.upsert.shuffle.parallelism") .defaultValue("200") diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index d9b01a10e56dd..f7191c586864d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; -import org.apache.hudi.client.HoodieInternalWriteStatusCoordinator; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -76,8 +75,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.SaveMode; -import scala.collection.JavaConverters; import java.io.IOException; import java.util.ArrayList; @@ -85,7 +82,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -142,7 +138,6 @@ public abstract HoodieData performClusteringWithRecordsRow(final Da final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata); - /** * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. * The number of new file groups created is bounded by numOutputGroups. @@ -161,32 +156,6 @@ public abstract HoodieData performClusteringWithRecordsRDD(final Ho final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata); - protected HoodieData performRowWrite(Dataset inputRecords, Map parameters) { - String uuid = UUID.randomUUID().toString(); - parameters.put(HoodieWriteConfig.BULKINSERT_ROW_IDENTIFY_ID.key(), uuid); - try { - inputRecords.write() - .format("hudi") - .options(JavaConverters.mapAsScalaMapConverter(parameters).asScala()) - .mode(SaveMode.Append) - .save(getWriteConfig().getBasePath()); - List writeStatusList = HoodieInternalWriteStatusCoordinator.get().getWriteStatuses(uuid) - .stream() - .map(internalWriteStatus -> { - WriteStatus status = new WriteStatus( - internalWriteStatus.isTrackSuccessRecords(), internalWriteStatus.getFailureFraction()); - status.setFileId(internalWriteStatus.getFileId()); - status.setTotalRecords(internalWriteStatus.getTotalRecords()); - status.setPartitionPath(internalWriteStatus.getPartitionPath()); - status.setStat(internalWriteStatus.getStat()); - return status; - }).collect(Collectors.toList()); - return getEngineContext().parallelize(writeStatusList); - } finally { - HoodieInternalWriteStatusCoordinator.get().removeStatuses(uuid); - } - } - protected BulkInsertPartitioner> getRowPartitioner(Map strategyParams, Schema schema) { return getPartitioner(strategyParams, schema, true); @@ -203,7 +172,7 @@ protected BulkInsertPartitioner>> getRDDPartitioner(Map< * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. * @param schema Schema of the data including metadata fields. */ - protected BulkInsertPartitioner getPartitioner(Map strategyParams, + private BulkInsertPartitioner getPartitioner(Map strategyParams, Schema schema, boolean isRowPartitioner) { Option orderByColumnsOpt = @@ -235,38 +204,6 @@ protected BulkInsertPartitioner getPartitioner(Map strate BulkInsertInternalPartitionerFactory.get(getWriteConfig().getBulkInsertSortMode())); } - /** - * Configure {@link BulkInsertPartitioner>} based on strategy params. - * - * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. - * @param parameters Map used to write the partitioner configurations. - * @return {@link RDDCustomColumnsSortPartitioner} if sort columns are provided, otherwise empty. - */ - protected void configRowPartitioner(Map strategyParams, Map parameters) { - Option.ofNullable(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key())) - .map(orderColumnStr -> { - parameters.put(HoodieWriteConfig.BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS.key(), orderColumnStr); - HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy = getWriteConfig().getLayoutOptimizationStrategy(); - switch (layoutOptStrategy) { - case ZORDER: - case HILBERT: - parameters.put(HoodieWriteConfig.BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME.key(), RowSpatialCurveSortPartitioner.class.getCanonicalName()); - break; - case LINEAR: - parameters.put(HoodieWriteConfig.BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME.key(), RowCustomColumnsSortPartitioner.class.getCanonicalName()); - break; - default: - throw new UnsupportedOperationException(String.format("Layout optimization strategy '%s' is not supported", layoutOptStrategy)); - } - parameters.compute(HoodieClusteringConfig.LAYOUT_OPTIMIZE_STRATEGY.key(), - (k, v) -> getWriteConfig().getString(HoodieClusteringConfig.LAYOUT_OPTIMIZE_STRATEGY)); - parameters.compute(HoodieClusteringConfig.LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD.key(), - (k, v) -> getWriteConfig().getString(HoodieClusteringConfig.LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD)); - return orderColumnStr; - }); - } - - /** * Submit job to execute clustering for the group with RDD APIs. */ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 8780519ea7971..c07024810017a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -184,7 +184,8 @@ object HoodieSparkSqlWriter { val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) = operation match { case WriteOperationType.DELETE => { - val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) // Convert to RDD[HoodieKey] + val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) + // Convert to RDD[HoodieKey] val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD() if (!tableExists) { diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java index e57695c03bf3c..9f878dfa572d2 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java @@ -20,8 +20,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; - import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper; + import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java index dc7f69e80a043..24157c694ef21 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java @@ -20,8 +20,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; - import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper; + import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; import org.apache.spark.sql.connector.write.WriterCommitMessage; From b8e848d0f8b32ff3c75762951e3af4c911419927 Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 5 Sep 2022 14:13:58 +0800 Subject: [PATCH 08/18] Fix error --- .../action/commit/BulkInsertDataInternalWriterHelper.java | 3 ++- .../hudi/table/action/commit/SparkBulkInsertRowWriter.java | 4 ++++ .../functional/TestHoodieSparkMergeOnReadTableClustering.java | 4 ++-- .../org/apache/hudi/functional/TestLayoutOptimization.scala | 2 +- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java index c15c4e647b6a9..4a41afa94413c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java @@ -126,7 +126,8 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo private Option getKeyGenerator(Properties properties) { TypedProperties typedProperties = new TypedProperties(); typedProperties.putAll(properties); - if (properties.get(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key()).equals(NonpartitionedKeyGenerator.class.getName())) { + if (Option.ofNullable(properties.get(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key())) + .map(v -> v.equals(NonpartitionedKeyGenerator.class.getName())).orElse(false)) { return Option.empty(); // Do not instantiate NonPartitionKeyGen } else { try { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java index a1a9d49811e37..1a3dbb4664a39 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java @@ -35,6 +35,10 @@ public class SparkBulkInsertRowWriter { + /** + * Perform bulk insert for {@link Dataset}, will not change timeline/index, return + * information about write files. + */ public static HoodieData bulkInsert(Dataset dataset, String instantTime, HoodieTable table, diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java index c10d17f346471..06114c2bf277d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -19,6 +19,7 @@ package org.apache.hudi.functional; +import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; @@ -247,8 +248,7 @@ private void doClusteringAndValidate(SparkRDDWriteClient client, HoodieTestDataGenerator dataGen, boolean clusteringAsRow) { if (clusteringAsRow) { - client.getConfig().setValue(HoodieClusteringConfig.CLUSTERING_AS_ROW, "true"); - client.getConfig().setAll(getPropertiesForKeyGen()); + client.getConfig().setValue(DataSourceWriteOptions.ENABLE_ROW_WRITER(), "true"); } client.cluster(clusteringCommitTime, true); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala index 29991378665e8..265a1588ce781 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala @@ -114,7 +114,7 @@ class TestLayoutOptimization extends HoodieClientTestBase { .option("hoodie.clustering.plan.strategy.small.file.limit", "629145600") .option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString) .option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 * 1024 * 1024L)) - .option(HoodieClusteringConfig.CLUSTERING_AS_ROW.key(), clusteringAsRow) + .option(DataSourceWriteOptions.ENABLE_ROW_WRITER.key(), clusteringAsRow) .option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_STRATEGY.key(), layoutOptimizationStrategy) .option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_SPATIAL_CURVE_BUILD_METHOD.key(), curveCompositionStrategy) .option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat,begin_lon") From 5a16d35ec42bf86e5759ebb155cad40e83aba9f9 Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 5 Sep 2022 14:58:17 +0800 Subject: [PATCH 09/18] Fix names --- .../MultipleSparkJobExecutionStrategy.java | 8 +++---- .../io/storage/row/HoodieRowCreateHandle.java | 24 +++++++------------ 2 files changed, 13 insertions(+), 19 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index f7191c586864d..78b266642efe3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -344,14 +344,14 @@ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, String[] baseFilePaths = clusteringOps .stream() .map(op -> { - ArrayList pairs = new ArrayList<>(); + ArrayList readPaths = new ArrayList<>(); if (op.getBootstrapFilePath() != null) { - pairs.add(op.getBootstrapFilePath()); + readPaths.add(op.getBootstrapFilePath()); } if (op.getDataFilePath() != null) { - pairs.add(op.getDataFilePath()); + readPaths.add(op.getDataFilePath()); } - return pairs; + return readPaths; }) .flatMap(Collection::stream) .filter(path -> !path.isEmpty()) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 648529575b811..3a9c2ed346710 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -172,21 +172,15 @@ private void writeRow(InternalRow row) { UTF8String recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); - InternalRow updatedRow; - if (preserveHoodieMetadata) { - updatedRow = new HoodieInternalRow(row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD), - row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD), - recordKey, - partitionPath, - fileName, row, true); - } else { - // This is the only meta-field that is generated dynamically, hence conversion b/w - // [[String]] and [[UTF8String]] is unavoidable - UTF8String seqId = UTF8String.fromString(seqIdGenerator.apply(GLOBAL_SEQ_NO.getAndIncrement())); - - updatedRow = new HoodieInternalRow(commitTime, seqId, recordKey, - partitionPath, fileName, row, true); - } + // This is the only meta-field that is generated dynamically, hence conversion b/w + // [[String]] and [[UTF8String]] is unavoidable if preserveHoodieMetadata is false + UTF8String seqId = preserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD) + : UTF8String.fromString(seqIdGenerator.apply(GLOBAL_SEQ_NO.getAndIncrement())); + UTF8String writeCommitTime = preserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD) + : commitTime; + + InternalRow updatedRow = new HoodieInternalRow(writeCommitTime, seqId, recordKey, + partitionPath, fileName, row, true); try { fileWriter.writeRow(recordKey, updatedRow); // NOTE: To avoid conversion on the hot-path we only convert [[UTF8String]] into [[String]] From 60ef51484364c4a7e8a0aa64817e96b4f5a277cf Mon Sep 17 00:00:00 2001 From: Rex An Date: Mon, 5 Sep 2022 15:03:41 +0800 Subject: [PATCH 10/18] Remove empty line --- .../org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 3a9c2ed346710..892a3be75483d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -171,7 +171,6 @@ private void writeRow(InternalRow row) { // over again) UTF8String recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); - // This is the only meta-field that is generated dynamically, hence conversion b/w // [[String]] and [[UTF8String]] is unavoidable if preserveHoodieMetadata is false UTF8String seqId = preserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD) From 1355882bd77d8bbbe0990a7fa07069255a10c7be Mon Sep 17 00:00:00 2001 From: Rex An Date: Fri, 16 Sep 2022 17:07:52 +0800 Subject: [PATCH 11/18] change read-path to directly create relation --- .../client/HoodieInternalWriteStatus.java | 9 ++ .../apache/hudi/JavaSparkAdaptorSupport.java | 59 ++++++++ .../MultipleSparkJobExecutionStrategy.java | 133 ++++++++++-------- .../SparkSingleFileSortExecutionStrategy.java | 29 ++-- .../SparkSortAndSizeExecutionStrategy.java | 33 ++--- .../RDDSpatialCurveSortPartitioner.java | 37 +---- .../RowSpatialCurveSortPartitioner.java | 50 ++----- .../SpatialCurveSortPartitionerBase.java | 83 +++++++++++ .../io/storage/row/HoodieRowCreateHandle.java | 10 +- .../HoodieSparkKeyGeneratorFactory.java | 17 +++ .../BulkInsertDataInternalWriterHelper.java | 8 +- .../commit/SparkBulkInsertRowWriter.java | 80 ----------- .../hudi/HoodieDatasetBulkInsertHelper.scala | 68 ++++++++- .../org/apache/hudi/HoodieUnsafeRDD.scala | 0 .../org/apache/hudi/SparkAdapterSupport.scala | 15 +- .../apache/spark/sql/HoodieUnsafeUtils.scala | 1 - .../apache/spark/sql/hudi/SparkAdapter.scala | 15 +- .../hudi/hadoop/HoodieROTablePathFilter.java | 13 +- .../apache/hudi/BaseFileOnlyRelation.scala | 5 +- .../org/apache/hudi/DataSourceOptions.scala | 19 +-- .../scala/org/apache/hudi/DefaultSource.scala | 86 ++++++----- .../sql/hudi/command/SqlKeyGenerator.scala | 2 +- .../hudi/functional/TestCOWDataSource.scala | 12 ++ .../spark/sql/adapter/Spark2Adapter.scala | 24 +++- .../spark/sql/adapter/BaseSpark3Adapter.scala | 20 ++- 25 files changed, 483 insertions(+), 345 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/SpatialCurveSortPartitionerBase.java delete mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java rename {hudi-spark-datasource/hudi-spark-common => hudi-client/hudi-spark-client}/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala (71%) rename {hudi-spark-datasource/hudi-spark-common => hudi-client/hudi-spark-client}/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala (100%) rename {hudi-spark-datasource/hudi-spark-common => hudi-client/hudi-spark-client}/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala (99%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java index 5a2449615121a..808eda5071e1f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java @@ -163,4 +163,13 @@ public String toString() { + totalRecords + ", errored Rows " + totalErrorRecords + ", global error " + (globalError != null); } + + public WriteStatus toWriteStatus() { + WriteStatus status = new WriteStatus(trackSuccessRecords, failureFraction); + status.setFileId(fileId); + status.setTotalRecords(totalRecords); + status.setPartitionPath(partitionPath); + status.setStat(stat); + return status; + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java new file mode 100644 index 0000000000000..1bfca6ac3b323 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java @@ -0,0 +1,59 @@ +/* + * 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; + +import org.apache.hudi.exception.HoodieException; +import org.apache.spark.sql.hudi.SparkAdapter; + +/** + * Java implementation to provide SparkAdapter when we need to adapt + * the difference between spark2 and spark3. + */ +public class JavaSparkAdaptorSupport { + + private JavaSparkAdaptorSupport() {} + + private static class AdapterSupport { + + private static final SparkAdapter ADAPTER = new AdapterSupport().sparkAdapter(); + + private SparkAdapter sparkAdapter() { + String adapterClass; + if (HoodieSparkUtils.isSpark3_3()) { + adapterClass = "org.apache.spark.sql.adapter.Spark3_3Adapter"; + } else if (HoodieSparkUtils.isSpark3_2()) { + adapterClass = "org.apache.spark.sql.adapter.Spark3_2Adapter"; + } else if (HoodieSparkUtils.isSpark3_0() || HoodieSparkUtils.isSpark3_1()) { + adapterClass = "org.apache.spark.sql.adapter.Spark3_1Adapter"; + } else { + adapterClass = "org.apache.spark.sql.adapter.Spark2Adapter"; + } + try { + return (SparkAdapter) this.getClass().getClassLoader().loadClass(adapterClass) + .newInstance(); + } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { + throw new HoodieException("Cannot instantiate SparkAdaptor", e); + } + } + } + + public static SparkAdapter sparkAdapter() { + return AdapterSupport.ADAPTER; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 78b266642efe3..7cd489638ccd1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -18,6 +18,7 @@ package org.apache.hudi.client.clustering.run.strategy; +import org.apache.hudi.JavaSparkAdaptorSupport; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -75,10 +76,13 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.sources.BaseRelation; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -103,19 +107,20 @@ public MultipleSparkJobExecutionStrategy(HoodieTable table, HoodieEngineContext @Override public HoodieWriteMetadata> performClustering(final HoodieClusteringPlan clusteringPlan, final Schema schema, final String instantTime) { JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(getEngineContext()); + boolean shouldPreserveMetadata = Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false); // execute clustering for each group async and collect WriteStatus Stream> writeStatusesStream = FutureUtils.allOf( clusteringPlan.getInputGroups().stream() .map(inputGroup -> { if (getWriteConfig().getBooleanOrDefault("hoodie.datasource.write.row.writer.enable", false)) { - return runClusteringForGroupAsyncWithRow(inputGroup, + return runClusteringForGroupAsyncAsRow(inputGroup, clusteringPlan.getStrategy().getStrategyParams(), - Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), + shouldPreserveMetadata, instantTime); } - return runClusteringForGroupAsyncWithRDD(inputGroup, + return runClusteringForGroupAsync(inputGroup, clusteringPlan.getStrategy().getStrategyParams(), - Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), + shouldPreserveMetadata, instantTime); }) .collect(Collectors.toList())) @@ -134,27 +139,35 @@ public HoodieWriteMetadata> performClustering(final Hood * Different from {@link performClusteringWithRecordsRDD}, this method take {@link Dataset} * as inputs. */ - public abstract HoodieData performClusteringWithRecordsRow(final Dataset inputRecords, final int numOutputGroups, final String instantTime, - final Map strategyParams, final Schema schema, - final List fileGroupIdList, final boolean preserveHoodieMetadata); + public abstract HoodieData performClusteringWithRecordsAsRow(final Dataset inputRecords, + final int numOutputGroups, + final String instantTime, + final Map strategyParams, + final Schema schema, + final List fileGroupIdList, + final boolean shouldPreserveHoodieMetadata); /** * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. * The number of new file groups created is bounded by numOutputGroups. * Note that commit is not done as part of strategy. commit is callers responsibility. * - * @param inputRecords RDD of {@link HoodieRecord}. - * @param numOutputGroups Number of output file groups. - * @param instantTime Clustering (replace commit) instant time. - * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. - * @param schema Schema of the data including metadata fields. - * @param fileGroupIdList File group id corresponding to each out group. - * @param preserveHoodieMetadata Whether to preserve commit metadata while clustering. + * @param inputRecords RDD of {@link HoodieRecord}. + * @param numOutputGroups Number of output file groups. + * @param instantTime Clustering (replace commit) instant time. + * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. + * @param schema Schema of the data including metadata fields. + * @param fileGroupIdList File group id corresponding to each out group. + * @param shouldPreserveHoodieMetadata Whether to preserve commit metadata while clustering. * @return RDD of {@link WriteStatus}. */ - public abstract HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, - final Map strategyParams, final Schema schema, - final List fileGroupIdList, final boolean preserveHoodieMetadata); + public abstract HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, + final int numOutputGroups, + final String instantTime, + final Map strategyParams, + final Schema schema, + final List fileGroupIdList, + final boolean shouldPreserveHoodieMetadata); protected BulkInsertPartitioner> getRowPartitioner(Map strategyParams, Schema schema) { @@ -173,8 +186,8 @@ protected BulkInsertPartitioner>> getRDDPartitioner(Map< * @param schema Schema of the data including metadata fields. */ private BulkInsertPartitioner getPartitioner(Map strategyParams, - Schema schema, - boolean isRowPartitioner) { + Schema schema, + boolean isRowPartitioner) { Option orderByColumnsOpt = Option.ofNullable(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key())) .map(listStr -> listStr.split(",")); @@ -186,17 +199,13 @@ private BulkInsertPartitioner getPartitioner(Map strategy case HILBERT: return isRowPartitioner ? new RowSpatialCurveSortPartitioner(getWriteConfig()) - : new RDDSpatialCurveSortPartitioner( - (HoodieSparkEngineContext) getEngineContext(), - orderByColumns, - layoutOptStrategy, - getWriteConfig().getLayoutOptimizationCurveBuildMethod(), - HoodieAvroUtils.addMetadataFields(schema)); + : new RDDSpatialCurveSortPartitioner((HoodieSparkEngineContext) getEngineContext(), orderByColumns, layoutOptStrategy, + getWriteConfig().getLayoutOptimizationCurveBuildMethod(), HoodieAvroUtils.addMetadataFields(schema)); case LINEAR: return isRowPartitioner ? new RowCustomColumnsSortPartitioner(orderByColumns) : new RDDCustomColumnsSortPartitioner(orderByColumns, HoodieAvroUtils.addMetadataFields(schema), - getWriteConfig().isConsistentLogicalTimestampEnabled()); + getWriteConfig().isConsistentLogicalTimestampEnabled()); default: throw new UnsupportedOperationException(String.format("Layout optimization strategy '%s' is not supported", layoutOptStrategy)); } @@ -205,13 +214,13 @@ private BulkInsertPartitioner getPartitioner(Map strategy } /** - * Submit job to execute clustering for the group with RDD APIs. + * Submit job to execute clustering for the group using Avro/HoodieRecord representation. */ - private CompletableFuture> runClusteringForGroupAsyncWithRDD(HoodieClusteringGroup clusteringGroup, Map strategyParams, - boolean preserveHoodieMetadata, String instantTime) { + private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams, + boolean preserveHoodieMetadata, String instantTime) { return CompletableFuture.supplyAsync(() -> { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); - HoodieData> inputRecords = readRecordsForGroupAsRDD(jsc, clusteringGroup, instantTime); + HoodieData> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) @@ -221,10 +230,12 @@ private CompletableFuture> runClusteringForGroupAsyncWit } /** - * Submit job to execute clustering for the group with dataset APIs. + * Submit job to execute clustering for the group, directly using the spark native Row representation. */ - private CompletableFuture> runClusteringForGroupAsyncWithRow(HoodieClusteringGroup clusteringGroup, Map strategyParams, - boolean preserveHoodieMetadata, String instantTime) { + private CompletableFuture> runClusteringForGroupAsyncAsRow(HoodieClusteringGroup clusteringGroup, + Map strategyParams, + boolean shouldPreserveHoodieMetadata, + String instantTime) { return CompletableFuture.supplyAsync(() -> { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); Dataset inputRecords = readRecordsForGroupAsRow(jsc, clusteringGroup, instantTime); @@ -232,14 +243,14 @@ private CompletableFuture> runClusteringForGroupAsyncWit List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) .collect(Collectors.toList()); - return performClusteringWithRecordsRow(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, preserveHoodieMetadata); + return performClusteringWithRecordsAsRow(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, shouldPreserveHoodieMetadata); }); } /** * Get RDD of all records for the group. This includes all records from file slice (Apply updates from log files, if any). */ - private HoodieData> readRecordsForGroupAsRDD(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { + private HoodieData> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); if (hasLogFiles) { @@ -334,14 +345,14 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex * Get dataset of all records for the group. This includes all records from file slice (Apply updates from log files, if any). */ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, - HoodieClusteringGroup clusteringGroup, - String instantTime) { + HoodieClusteringGroup clusteringGroup, + String instantTime) { List clusteringOps = clusteringGroup.getSlices().stream() .map(ClusteringOperation::create).collect(Collectors.toList()); boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); SQLContext sqlContext = new SQLContext(jsc.sc()); - String[] baseFilePaths = clusteringOps + Path[] baseFilePaths = clusteringOps .stream() .map(op -> { ArrayList readPaths = new ArrayList<>(); @@ -355,33 +366,37 @@ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, }) .flatMap(Collection::stream) .filter(path -> !path.isEmpty()) - .toArray(String[]::new); - String[] deltaPaths = clusteringOps - .stream() - .filter(op -> !op.getDeltaFilePaths().isEmpty()) - .flatMap(op -> op.getDeltaFilePaths().stream()) - .toArray(String[]::new); + .map(Path::new) + .toArray(Path[]::new); + + HashMap params = new HashMap<>(); + params.put("hoodie.datasource.query.type", "snapshot"); + params.put("as.of.instant", instantTime); - Dataset inputRecords; + Path[] paths; if (hasLogFiles) { String compactionFractor = Option.ofNullable(getWriteConfig().getString("compaction.memory.fraction")) .orElse("0.75"); - String[] paths = CollectionUtils.combine(baseFilePaths, deltaPaths); - inputRecords = sqlContext.read() - .format("org.apache.hudi") - .option("hoodie.datasource.query.type", "snapshot") - .option("compaction.memory.fraction", compactionFractor) - .option("as.of.instant", instantTime) - .option("hoodie.datasource.read.paths", String.join(",", paths)) - .load(); + params.put("compaction.memory.fraction", compactionFractor); + + Path[] deltaPaths = clusteringOps + .stream() + .filter(op -> !op.getDeltaFilePaths().isEmpty()) + .flatMap(op -> op.getDeltaFilePaths().stream()) + .map(Path::new) + .toArray(Path[]::new); + paths = CollectionUtils.combine(baseFilePaths, deltaPaths); } else { - inputRecords = sqlContext.read() - .format("org.apache.hudi") - .option("as.of.instant", instantTime) - .option("hoodie.datasource.read.paths", String.join(",", baseFilePaths)) - .load(); + paths = baseFilePaths; } - return inputRecords; + + // Building HoodieFileIndex needs this param to decide query path + params.put("glob.paths", String.join(",", Arrays.stream(paths).map(Path::toString).toArray(String[]::new))); + + // Let Hudi relations to fetch the schema from the table itself + BaseRelation relation = JavaSparkAdaptorSupport.sparkAdapter() + .createRelation(getHoodieTable().getMetaClient(), sqlContext, null, paths, params); + return sqlContext.baseRelationToDataFrame(relation); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index 20145b343635c..48d1f755f7062 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -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; @@ -33,7 +34,6 @@ import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; import org.apache.avro.Schema; -import org.apache.hudi.table.action.commit.SparkBulkInsertRowWriter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.sql.Dataset; @@ -58,13 +58,13 @@ public SparkSingleFileSortExecutionStrategy(HoodieTable table, } @Override - public HoodieData performClusteringWithRecordsRow(Dataset inputRecords, - int numOutputGroups, - String instantTime, - Map strategyParams, - Schema schema, - List fileGroupIdList, - boolean preserveHoodieMetadata) { + public HoodieData performClusteringWithRecordsAsRow(Dataset inputRecords, + int numOutputGroups, + String instantTime, + Map strategyParams, + Schema schema, + List fileGroupIdList, + boolean shouldPreserveHoodieMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } @@ -74,15 +74,10 @@ public HoodieData performClusteringWithRecordsRow(Dataset inpu .withBulkInsertParallelism(numOutputGroups) .withProps(getWriteConfig().getProps()).build(); - boolean shouldPreserveHoodieMetadata = preserveHoodieMetadata; - if (!newConfig.populateMetaFields() && preserveHoodieMetadata) { - LOG.warn("Will setting preserveHoodieMetadata to false as populateMetaFields is false"); - shouldPreserveHoodieMetadata = false; - } - + // 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)); - return SparkBulkInsertRowWriter.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata); } @@ -93,7 +88,7 @@ public HoodieData performClusteringWithRecordsRDD(HoodieData strategyParams, Schema schema, List fileGroupIdList, - boolean preserveHoodieMetadata) { + boolean shouldPreserveHoodieMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } @@ -106,6 +101,6 @@ public HoodieData performClusteringWithRecordsRDD(HoodieData) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, - false, getRDDPartitioner(strategyParams, schema), true, numOutputGroups, new SingleFileHandleCreateFactory(fileGroupIdList.get(0).getFileId(), preserveHoodieMetadata)); + false, getRDDPartitioner(strategyParams, schema), true, numOutputGroups, new SingleFileHandleCreateFactory(fileGroupIdList.get(0).getFileId(), shouldPreserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 33ad16f909c14..c48432c0b0898 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -18,6 +18,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; @@ -31,7 +32,6 @@ import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; import org.apache.avro.Schema; -import org.apache.hudi.table.action.commit.SparkBulkInsertRowWriter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.sql.Dataset; @@ -56,37 +56,38 @@ public SparkSortAndSizeExecutionStrategy(HoodieTable table, } @Override - public HoodieData performClusteringWithRecordsRow(Dataset inputRecords, int numOutputGroups, - String instantTime, Map strategyParams, Schema schema, - List fileGroupIdList, boolean preserveHoodieMetadata) { + public HoodieData performClusteringWithRecordsAsRow(Dataset inputRecords, + int numOutputGroups, + String instantTime, Map strategyParams, + Schema schema, + List fileGroupIdList, + boolean shouldPreserveHoodieMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() .withBulkInsertParallelism(numOutputGroups) .withProps(getWriteConfig().getProps()).build(); - boolean shouldPreserveHoodieMetadata = preserveHoodieMetadata; - if (!newConfig.populateMetaFields() && preserveHoodieMetadata) { - LOG.warn("Will setting preserveHoodieMetadata to false as populateMetaFields is false"); - shouldPreserveHoodieMetadata = false; - } - newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); - return SparkBulkInsertRowWriter.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + return HoodieDatasetBulkInsertHelper.bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, getRowPartitioner(strategyParams, schema), numOutputGroups, shouldPreserveHoodieMetadata); } @Override - public HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, - final String instantTime, final Map strategyParams, final Schema schema, - final List fileGroupIdList, final boolean preserveHoodieMetadata) { + public HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, + final int numOutputGroups, + final String instantTime, + final Map strategyParams, + final Schema schema, + final List fileGroupIdList, + final boolean shouldPreserveHoodieMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() .withBulkInsertParallelism(numOutputGroups) .withProps(getWriteConfig().getProps()).build(); newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); - return (HoodieData) SparkBulkInsertHelper.newInstance() - .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getRDDPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); + return (HoodieData) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), + newConfig, false, getRDDPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(shouldPreserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java index 50a0a534f881b..2ab9107fa54b5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java @@ -29,8 +29,6 @@ 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; @@ -38,32 +36,24 @@ 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 HoodieRecordPayload type */ public class RDDSpatialCurveSortPartitioner - implements BulkInsertPartitioner>> { + extends SpatialCurveSortPartitionerBase>> { 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> repartitionRecords(JavaRDD> reco return hoodieRecord; }); } - - private Dataset reorder(Dataset dataset, int numOutputGroups) { - if (orderByColumns.length == 0) { - // No-op - return dataset; - } - - List 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; - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java index 7f9a9ac07da6a..1217477c9d817 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RowSpatialCurveSortPartitioner.java @@ -20,57 +20,23 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.sort.SpaceCurveSortingHelper; -import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import java.util.Arrays; -import java.util.List; - -public class RowSpatialCurveSortPartitioner implements BulkInsertPartitioner> { - - private final String[] orderByColumns; - private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy; - private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType; +public class RowSpatialCurveSortPartitioner extends SpatialCurveSortPartitionerBase> { public RowSpatialCurveSortPartitioner(HoodieWriteConfig config) { - this.layoutOptStrategy = config.getLayoutOptimizationStrategy(); - if (config.getClusteringSortColumns() != null) { - this.orderByColumns = Arrays.stream(config.getClusteringSortColumns().split(",")) - .map(String::trim).toArray(String[]::new); - } else { - throw new IllegalArgumentException("The config " - + HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key() + " must be provided"); - } - this.curveCompositionStrategyType = config.getLayoutOptimizationCurveBuildMethod(); + super(config.getClusteringSortColumns(), config.getLayoutOptimizationStrategy(), config.getLayoutOptimizationCurveBuildMethod()); } - @Override - public Dataset repartitionRecords(Dataset records, int outputPartitions) { - return reorder(records, outputPartitions); - } - - private Dataset reorder(Dataset dataset, int numOutputGroups) { - if (orderByColumns.length == 0) { - // No-op - return dataset; - } - - List 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)); - } + public RowSpatialCurveSortPartitioner(String[] orderByColumns, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType) { + super(orderByColumns, layoutOptStrategy, curveCompositionStrategyType); } @Override - public boolean arePartitionRecordsSorted() { - return true; + public Dataset repartitionRecords(Dataset records, int outputPartitions) { + return reorder(records, outputPartitions); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/SpatialCurveSortPartitionerBase.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/SpatialCurveSortPartitionerBase.java new file mode 100644 index 0000000000000..96048f2782bc1 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/SpatialCurveSortPartitionerBase.java @@ -0,0 +1,83 @@ +/* + * 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.sort.SpaceCurveSortingHelper; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import java.util.Arrays; +import java.util.List; + +public abstract class SpatialCurveSortPartitionerBase implements BulkInsertPartitioner { + + private final String[] orderByColumns; + private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy; + private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType; + + public SpatialCurveSortPartitionerBase(String orderByColumns, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType) { + if (orderByColumns != null) { + this.orderByColumns = Arrays.stream(orderByColumns.split(",")) + .map(String::trim).toArray(String[]::new); + } else { + throw new IllegalArgumentException("The config " + + HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key() + " must be provided"); + } + this.layoutOptStrategy = layoutOptStrategy; + this.curveCompositionStrategyType = curveCompositionStrategyType; + } + + public SpatialCurveSortPartitionerBase(String[] orderByColumns, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType) { + this.orderByColumns = orderByColumns; + this.layoutOptStrategy = layoutOptStrategy; + this.curveCompositionStrategyType = curveCompositionStrategyType; + } + + /** + * Mapping specified multi need-to-order columns to one dimension while preserving data locality. + */ + protected Dataset reorder(Dataset dataset, int numOutputGroups) { + if (orderByColumns.length == 0) { + // No-op + return dataset; + } + + List 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; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 892a3be75483d..9da04f72600b7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -70,7 +70,7 @@ public class HoodieRowCreateHandle implements Serializable { private final UTF8String commitTime; private final Function seqIdGenerator; - private final boolean preserveHoodieMetadata; + private final boolean shouldPreserveHoodieMetadata; private final HoodieTimer currTimer; @@ -99,7 +99,7 @@ public HoodieRowCreateHandle(HoodieTable table, long taskId, long taskEpochId, StructType structType, - boolean preserveHoodieMetadata) { + boolean shouldPreserveHoodieMetadata) { this.partitionPath = partitionPath; this.table = table; this.writeConfig = writeConfig; @@ -120,7 +120,7 @@ public HoodieRowCreateHandle(HoodieTable table, this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(), writeConfig.getWriteStatusFailureFraction()); - this.preserveHoodieMetadata = preserveHoodieMetadata; + this.shouldPreserveHoodieMetadata = shouldPreserveHoodieMetadata; writeStatus.setPartitionPath(partitionPath); writeStatus.setFileId(fileId); @@ -173,9 +173,9 @@ private void writeRow(InternalRow row) { UTF8String partitionPath = row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); // This is the only meta-field that is generated dynamically, hence conversion b/w // [[String]] and [[UTF8String]] is unavoidable if preserveHoodieMetadata is false - UTF8String seqId = preserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD) + UTF8String seqId = shouldPreserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD_ORD) : UTF8String.fromString(seqIdGenerator.apply(GLOBAL_SEQ_NO.getAndIncrement())); - UTF8String writeCommitTime = preserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD) + UTF8String writeCommitTime = shouldPreserveHoodieMetadata ? row.getUTF8String(HoodieRecord.COMMIT_TIME_METADATA_FIELD_ORD) : commitTime; InternalRow updatedRow = new HoodieInternalRow(writeCommitTime, seqId, recordKey, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java index 165b27d6ce283..10141d74af447 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java @@ -30,6 +30,7 @@ import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.keygen.TimestampBasedKeyGenerator; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; import org.slf4j.Logger; @@ -75,6 +76,22 @@ public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx } } + public static String inferKeyGenClazz(TypedProperties props) { + String partitionFields = props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), null); + if (partitionFields != null) { + int numPartFields = partitionFields.split(",").length; + String recordsKeyFields = props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()); + int numRecordKeyFields = recordsKeyFields.split(",").length; + if (numPartFields == 1 && numRecordKeyFields == 1) { + return SimpleKeyGenerator.class.getName(); + } else { + return ComplexKeyGenerator.class.getName(); + } + } else { + return NonpartitionedKeyGenerator.class.getName(); + } + } + public static String getKeyGeneratorClassName(TypedProperties props) { String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), null); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java index 4a41afa94413c..12e9dda81a5bc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertDataInternalWriterHelper.java @@ -65,7 +65,7 @@ public class BulkInsertDataInternalWriterHelper { private final String fileIdPrefix; private final Map handles = new HashMap<>(); private final boolean populateMetaFields; - private final boolean preserveHoodieMetadata; + private final boolean shouldPreserveHoodieMetadata; private final Option keyGeneratorOpt; private final boolean simpleKeyGen; private final int simplePartitionFieldIndex; @@ -87,7 +87,7 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, - boolean populateMetaFields, boolean arePartitionRecordsSorted, boolean preserveHoodieMetadata) { + boolean populateMetaFields, boolean arePartitionRecordsSorted, boolean shouldPreserveHoodieMetadata) { this.hoodieTable = hoodieTable; this.writeConfig = writeConfig; this.instantTime = instantTime; @@ -96,7 +96,7 @@ public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteCo this.taskEpochId = taskEpochId; this.structType = structType; this.populateMetaFields = populateMetaFields; - this.preserveHoodieMetadata = preserveHoodieMetadata; + this.shouldPreserveHoodieMetadata = shouldPreserveHoodieMetadata; this.arePartitionRecordsSorted = arePartitionRecordsSorted; this.fileIdPrefix = UUID.randomUUID().toString(); @@ -208,7 +208,7 @@ private HoodieRowCreateHandle getRowCreateHandle(String partitionPath) throws IO private HoodieRowCreateHandle createHandle(String partitionPath) { return new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(), - instantTime, taskPartitionId, taskId, taskEpochId, structType, preserveHoodieMetadata); + instantTime, taskPartitionId, taskId, taskEpochId, structType, shouldPreserveHoodieMetadata); } private String getNextFileId() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java deleted file mode 100644 index 1a3dbb4664a39..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertRowWriter.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.table.action.commit; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.data.HoodieData; -import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; - -import java.util.Iterator; -import java.util.List; - -public class SparkBulkInsertRowWriter { - - /** - * Perform bulk insert for {@link Dataset}, will not change timeline/index, return - * information about write files. - */ - public static HoodieData bulkInsert(Dataset dataset, - String instantTime, - HoodieTable table, - HoodieWriteConfig writeConfig, - BulkInsertPartitioner> partitioner, - int parallelism, - boolean preserveHoodieMetadata) { - Dataset repartitionedDataset = partitioner.repartitionRecords(dataset, parallelism); - - boolean arePartitionRecordsSorted = partitioner.arePartitionRecordsSorted(); - StructType schema = dataset.schema(); - List writeStatuses = repartitionedDataset.queryExecution().toRdd().toJavaRDD().mapPartitions( - (FlatMapFunction, WriteStatus>) rowIterator -> { - TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier(); - int taskPartitionId = taskContextSupplier.getPartitionIdSupplier().get(); - long taskId = taskContextSupplier.getStageIdSupplier().get(); - long taskEpochId = taskContextSupplier.getAttemptIdSupplier().get(); - - final BulkInsertDataInternalWriterHelper writer = - new BulkInsertDataInternalWriterHelper(table, writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, - schema, writeConfig.populateMetaFields(), arePartitionRecordsSorted, preserveHoodieMetadata); - while (rowIterator.hasNext()) { - writer.write(rowIterator.next()); - } - return writer.getWriteStatuses() - .stream() - .map(internalWriteStatus -> { - WriteStatus status = new WriteStatus( - internalWriteStatus.isTrackSuccessRecords(), internalWriteStatus.getFailureFraction()); - status.setFileId(internalWriteStatus.getFileId()); - status.setTotalRecords(internalWriteStatus.getTotalRecords()); - status.setPartitionPath(internalWriteStatus.getPartitionPath()); - status.setStat(internalWriteStatus.getStat()); - return status; - }).iterator(); - }).collect(); - return table.getContext().parallelize(writeStatuses); - } -} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala similarity index 71% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 09f1fac2c874d..15f5dd019cced 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -17,14 +17,19 @@ package org.apache.hudi +import org.apache.hudi.client.WriteStatus import org.apache.hudi.client.model.HoodieInternalRow import org.apache.hudi.common.config.TypedProperties -import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.data.HoodieData +import org.apache.hudi.common.engine.TaskContextSupplier +import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload} import org.apache.hudi.common.util.ReflectionUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.index.SparkHoodieIndexFactory +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{BuiltinKeyGenerator, SparkKeyGeneratorInterface} -import org.apache.hudi.table.BulkInsertPartitioner +import org.apache.hudi.table.{BulkInsertPartitioner, HoodieTable} +import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue} @@ -33,8 +38,7 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.sql.{DataFrame, Dataset, HoodieUnsafeUtils, Row} import org.apache.spark.unsafe.types.UTF8String -import scala.collection.JavaConverters.asScalaBufferConverter -import scala.collection.mutable +import scala.collection.JavaConverters.{asScalaBufferConverter, seqAsJavaListConverter} object HoodieDatasetBulkInsertHelper extends Logging { @@ -55,8 +59,7 @@ object HoodieDatasetBulkInsertHelper extends Logging { val populateMetaFields = config.populateMetaFields() val schema = df.schema - val keyGeneratorClassName = config.getStringOrThrow(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME, - "Key-generator class name is required") + val keyGeneratorClassName = getKeyGenClassName(config) val prependedRdd: RDD[InternalRow] = df.queryExecution.toRdd.mapPartitions { iter => @@ -105,6 +108,57 @@ object HoodieDatasetBulkInsertHelper extends Logging { partitioner.repartitionRecords(trimmedDF, config.getBulkInsertShuffleParallelism) } + /** + * Perform bulk insert for [[Dataset]], will not change timeline/index, return + * information about write files. + */ + def bulkInsert(dataset: Dataset[Row], + instantTime: String, + table: HoodieTable[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]], _, _, _], + writeConfig: HoodieWriteConfig, + partitioner: BulkInsertPartitioner[Dataset[Row]], + parallelism: Int, + shouldPreserveHoodieMetadata: Boolean): HoodieData[WriteStatus] = { + val repartitionedDataset = partitioner.repartitionRecords(dataset, parallelism) + val arePartitionRecordsSorted = partitioner.arePartitionRecordsSorted + val schema = dataset.schema + val writeStatuses = repartitionedDataset.queryExecution.toRdd.mapPartitions(iter => { + val taskContextSupplier: TaskContextSupplier = table.getTaskContextSupplier + val taskPartitionId = taskContextSupplier.getPartitionIdSupplier.get + val taskId = taskContextSupplier.getStageIdSupplier.get.toLong + val taskEpochId = taskContextSupplier.getAttemptIdSupplier.get + val writer = new BulkInsertDataInternalWriterHelper( + table, + writeConfig, + instantTime, + taskPartitionId, + taskId, + taskEpochId, + schema, + writeConfig.populateMetaFields, + arePartitionRecordsSorted, + shouldPreserveHoodieMetadata) + + try { + iter.foreach(writer.write) + } catch { + case t: Throwable => + writer.abort() + throw t + } finally { + writer.close() + } + + writer.getWriteStatuses.asScala.map(_.toWriteStatus).iterator + }).collect() + table.getContext.parallelize(writeStatuses.toList.asJava) + } + + private def getKeyGenClassName(config: HoodieWriteConfig): String = { + Option(config.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key())) + .getOrElse(HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(config.getProps)) + } + private def dedupeRows(rdd: RDD[InternalRow], schema: StructType, preCombineFieldRef: String, isGlobalIndex: Boolean): RDD[InternalRow] = { val recordKeyMetaFieldOrd = schema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD) val partitionPathMetaFieldOrd = schema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD) @@ -149,7 +203,7 @@ object HoodieDatasetBulkInsertHelper extends Logging { } private def getPartitionPathFields(config: HoodieWriteConfig): Seq[String] = { - val keyGeneratorClassName = config.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME) + val keyGeneratorClassName = getKeyGenClassName(config) val keyGenerator = ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps)).asInstanceOf[BuiltinKeyGenerator] keyGenerator.getPartitionPathFields.asScala } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala similarity index 100% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala index 6d55309779ce4..b0eac3d956fbc 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala @@ -26,17 +26,6 @@ import org.apache.spark.sql.hudi.SparkAdapter */ trait SparkAdapterSupport { - lazy val sparkAdapter: SparkAdapter = { - val adapterClass = if (HoodieSparkUtils.isSpark3_3) { - "org.apache.spark.sql.adapter.Spark3_3Adapter" - } else if (HoodieSparkUtils.isSpark3_2) { - "org.apache.spark.sql.adapter.Spark3_2Adapter" - } else if (HoodieSparkUtils.isSpark3_0 || HoodieSparkUtils.isSpark3_1) { - "org.apache.spark.sql.adapter.Spark3_1Adapter" - } else { - "org.apache.spark.sql.adapter.Spark2Adapter" - } - getClass.getClassLoader.loadClass(adapterClass) - .newInstance().asInstanceOf[SparkAdapter] - } + lazy val sparkAdapter: SparkAdapter = JavaSparkAdaptorSupport.sparkAdapter() + } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala similarity index 99% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala index bd7f2f54560e8..edf05f2db2ec9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala @@ -84,5 +84,4 @@ object HoodieUnsafeUtils { .map(p => p._1) .collect() } - } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index d78216a48e8b4..a5ae82f146f80 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.hudi import org.apache.avro.Schema +import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConverters, HoodieAvroSerializer} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -32,11 +34,13 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, Subque import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, LogicalRelation, PartitionedFile, SparkParsePartitionUtil} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SparkSession} +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SQLContext, SparkSession} import org.apache.spark.storage.StorageLevel import java.util.Locale +import java.util.{Map => JMap} /** * Interface adapting discrepancies and incompatibilities between different Spark versions @@ -141,6 +145,15 @@ trait SparkAdapter extends Serializable { */ def createInterpretedPredicate(e: Expression): InterpretedPredicate + /** + * Create Hoodie relation based on globPaths, otherwise use tablePath if it's empty + */ + def createRelation(metaClient: HoodieTableMetaClient, + sqlContext: SQLContext, + schema: Schema, + globPaths: Array[Path], + parameters: JMap[String, String]): BaseRelation + /** * Create instance of [[HoodieFileScanRDD]] * SPARK-37273 FileScanRDD constructor changed in SPARK 3.3 diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java index be868ad29b284..44844a8d475ec 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java @@ -183,8 +183,17 @@ public boolean accept(Path path) { metaClientCache.put(baseDir.toString(), metaClient); } - fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, - metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf())); + if (getConf().get("as.of.instant") != null) { + // Build FileSystemViewManager with specified time, it's necessary to set this config when you may + // access old version files. For example, in spark side, using "hoodie.datasource.read.paths" + // which contains old version files, if not specify this value, these files will be filtered. + fsView = FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, + metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf()), + metaClient.getActiveTimeline().filterCompletedInstants().findInstantsBeforeOrEquals(getConf().get("as.of.instant"))); + } else { + fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, + metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf())); + } String partition = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), folder); List latestFiles = fsView.getLatestBaseFiles(partition).collect(Collectors.toList()); // populate the cache diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala index 119c61f84bc19..ba38dec812a30 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala @@ -160,9 +160,6 @@ class BaseFileOnlyRelation(sqlContext: SQLContext, fileFormat = fileFormat, optParams)(sparkSession) } else { - val readPathsStr = optParams.get(DataSourceReadOptions.READ_PATHS.key) - val extraReadPaths = readPathsStr.map(p => p.split(",").toSeq).getOrElse(Seq()) - // NOTE: Spark is able to infer partitioning values from partition path only when Hive-style partitioning // scheme is used. Therefore, we fallback to reading the table as non-partitioned (specifying // partitionColumns = Seq.empty) whenever Hive-style partitioning is not involved @@ -174,7 +171,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext, DataSource.apply( sparkSession = sparkSession, - paths = extraReadPaths, + paths = globPaths.map(_.toString), // Here we should specify the schema to the latest commit schema since // the table schema evolution. userSpecifiedSchema = userSchema.orElse(Some(tableStructSchema)), diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 16f52f33b13c5..aed483c5f1cc8 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -28,6 +28,7 @@ import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.keygen.constant.KeyGeneratorOptions +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.hudi.sync.common.util.ConfigUtils @@ -326,7 +327,7 @@ object DataSourceWriteOptions { * Key generator class, that implements will extract the key out of incoming record. */ val keyGeneraterInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { - Option.of(DataSourceOptionsHelper.inferKeyGenClazz(p.getProps)) + Option.of(HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(p.getProps)) }) val KEYGENERATOR_CLASS_NAME: ConfigProperty[String] = ConfigProperty @@ -784,22 +785,6 @@ object DataSourceOptionsHelper { ) ++ translateConfigurations(paramsWithGlobalProps) } - def inferKeyGenClazz(props: TypedProperties): String = { - val partitionFields = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), null) - if (partitionFields != null) { - val numPartFields = partitionFields.split(",").length - val recordsKeyFields = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD.key(), DataSourceWriteOptions.RECORDKEY_FIELD.defaultValue()) - val numRecordKeyFields = recordsKeyFields.split(",").length - if (numPartFields == 1 && numRecordKeyFields == 1) { - classOf[SimpleKeyGenerator].getName - } else { - classOf[ComplexKeyGenerator].getName - } - } else { - classOf[NonpartitionedKeyGenerator].getName - } - } - implicit def scalaFunctionToJavaFunction[From, To](function: (From) => To): JavaFunction[From, To] = { new JavaFunction[From, To] { override def apply (input: From): To = function (input) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index d5cbf020ed3e5..fc9c5cd620a93 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -110,44 +110,9 @@ class DefaultSource extends RelationProvider val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent val tableType = metaClient.getTableType val queryType = parameters(QUERY_TYPE.key) - // NOTE: In cases when Hive Metastore is used as catalog and the table is partitioned, schema in the HMS might contain - // Hive-specific partitioning columns created specifically for HMS to handle partitioning appropriately. In that - // case we opt in to not be providing catalog's schema, and instead force Hudi relations to fetch the schema - // from the table itself - val userSchema = if (isUsingHiveCatalog(sqlContext.sparkSession)) { - None - } else { - Option(schema) - } - log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType") - if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) { - new EmptyRelation(sqlContext, metaClient) - } else { - (tableType, queryType, isBootstrappedTable) match { - case (COPY_ON_WRITE, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) | - (COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) | - (MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) => - resolveBaseFileOnlyRelation(sqlContext, globPaths, userSchema, metaClient, parameters) - - case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) => - new IncrementalRelation(sqlContext, parameters, userSchema, metaClient) - - case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) => - new MergeOnReadSnapshotRelation(sqlContext, parameters, userSchema, globPaths, metaClient) - - case (MERGE_ON_READ, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) => - new MergeOnReadIncrementalRelation(sqlContext, parameters, userSchema, metaClient) - - case (_, _, true) => - new HoodieBootstrapRelation(sqlContext, userSchema, globPaths, metaClient, parameters) - - case (_, _, _) => - throw new HoodieException(s"Invalid query type : $queryType for tableType: $tableType," + - s"isBootstrappedTable: $isBootstrappedTable ") - } - } + DefaultSource.createRelation(metaClient, sqlContext, schema, globPaths, parameters) } def getValidCommits(metaClient: HoodieTableMetaClient): String = { @@ -228,6 +193,55 @@ class DefaultSource extends RelationProvider parameters: Map[String, String]): Source = { new HoodieStreamSource(sqlContext, metadataPath, schema, parameters) } +} + +object DefaultSource { + + def createRelation(metaClient: HoodieTableMetaClient, + sqlContext: SQLContext, + schema: StructType, + globPaths: Seq[Path], + parameters: Map[String, String]): BaseRelation = { + val tableType = metaClient.getTableType + val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent + val queryType = parameters(QUERY_TYPE.key) + // NOTE: In cases when Hive Metastore is used as catalog and the table is partitioned, schema in the HMS might contain + // Hive-specific partitioning columns created specifically for HMS to handle partitioning appropriately. In that + // case we opt in to not be providing catalog's schema, and instead force Hudi relations to fetch the schema + // from the table itself + val userSchema = if (isUsingHiveCatalog(sqlContext.sparkSession)) { + None + } else { + Option(schema) + } + + if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) { + new EmptyRelation(sqlContext, metaClient) + } else { + (tableType, queryType, isBootstrappedTable) match { + case (COPY_ON_WRITE, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) | + (COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) | + (MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) => + resolveBaseFileOnlyRelation(sqlContext, globPaths, userSchema, metaClient, parameters) + + case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) => + new IncrementalRelation(sqlContext, parameters, userSchema, metaClient) + + case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) => + new MergeOnReadSnapshotRelation(sqlContext, parameters, userSchema, globPaths, metaClient) + + case (MERGE_ON_READ, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) => + new MergeOnReadIncrementalRelation(sqlContext, parameters, userSchema, metaClient) + + case (_, _, true) => + new HoodieBootstrapRelation(sqlContext, userSchema, globPaths, metaClient, parameters) + + case (_, _, _) => + throw new HoodieException(s"Invalid query type : $queryType for tableType: $tableType," + + s"isBootstrappedTable: $isBootstrappedTable ") + } + } + } private def resolveBaseFileOnlyRelation(sqlContext: SQLContext, globPaths: Seq[Path], diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala index 798ed84b0939c..ca160ff7d59b1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala @@ -136,7 +136,7 @@ object SqlKeyGenerator { if (beforeKeyGenClassName != null && beforeKeyGenClassName.nonEmpty) { HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(beforeKeyGenClassName) } else { - DataSourceOptionsHelper.inferKeyGenClazz(props) + HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(props) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index c7eef5bce4270..e1f713389c524 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -306,6 +306,10 @@ class TestCOWDataSource extends HoodieClientTestBase { .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) + val metaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true).build() + + val instantTime = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.findFirst().get().getTimestamp val record1FilePaths = fs.listStatus(new Path(basePath, dataGen.getPartitionPaths.head)) .filter(!_.getPath.getName.contains("hoodie_partition_metadata")) @@ -316,6 +320,13 @@ class TestCOWDataSource extends HoodieClientTestBase { val records2 = dataGen.generateInsertsContainsAllPartitions("002", 20) val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + val inputDF3 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) + inputDF3.write.format("org.apache.hudi") .options(commonOpts) // Use bulk insert here to make sure the files have different file groups. .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) @@ -323,6 +334,7 @@ class TestCOWDataSource extends HoodieClientTestBase { .save(basePath) val hudiReadPathDF = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key(), instantTime) .option(DataSourceReadOptions.READ_PATHS.key, record1FilePaths) .load() diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index eabdd337ee5d8..2ed6b1c2022d4 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -19,8 +19,13 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema -import org.apache.hudi.{Spark2HoodieFileScanRDD, Spark2RowSerDe} +import org.apache.hadoop.fs.Path +import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL} +import org.apache.hudi.{AvroConversionUtils, BaseFileOnlyRelation, DefaultSource, EmptyRelation, HoodieBootstrapRelation, IncrementalRelation, MergeOnReadIncrementalRelation, MergeOnReadSnapshotRelation, Spark2HoodieFileScanRDD, Spark2RowSerDe} import org.apache.hudi.client.utils.SparkRowSerDe +import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.avro._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -28,19 +33,23 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.logical.{Command, InsertIntoTable, Join, LogicalPlan, DeleteFromTable} +import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, InsertIntoTable, Join, LogicalPlan} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile, Spark2ParsePartitionUtil, SparkParsePartitionUtil} +import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark2CatalystExpressionUtils, HoodieSpark2CatalystPlanUtils, Row, SparkSession} +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark2CatalystExpressionUtils, HoodieSpark2CatalystPlanUtils, Row, SQLContext, SparkSession} import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters.mapAsScalaMapConverter +import java.util.{Map => JMap} /** * Implementation of [[SparkAdapter]] for Spark 2.4.x @@ -123,6 +132,15 @@ class Spark2Adapter extends SparkAdapter { InterpretedPredicate.create(e) } + override def createRelation(metaClient: HoodieTableMetaClient, + sqlContext: SQLContext, + schema: Schema, + globPaths: Array[Path], + parameters: JMap[String, String]): BaseRelation = { + val dataSchema = Option(schema).map(AvroConversionUtils.convertAvroSchemaToStructType).orNull + DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) + } + override def createHoodieFileScanRDD(sparkSession: SparkSession, readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index 44974790b32ab..51f6429b44e51 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -17,8 +17,12 @@ package org.apache.spark.sql.adapter -import org.apache.hudi.Spark3RowSerDe +import org.apache.avro.Schema +import org.apache.hadoop.fs.Path +import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark3RowSerDe} import org.apache.hudi.client.utils.SparkRowSerDe +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.exception.HoodieException import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -31,10 +35,13 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.{HoodieCatalystPlansUtils, HoodieSpark3CatalystPlanUtils, Row, SparkSession} +import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.{HoodieCatalystPlansUtils, HoodieSpark3CatalystPlanUtils, Row, SQLContext, SparkSession} import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.{DISK_ONLY, DISK_ONLY_2, DISK_ONLY_3, MEMORY_AND_DISK, MEMORY_AND_DISK_2, MEMORY_AND_DISK_SER, MEMORY_AND_DISK_SER_2, MEMORY_ONLY, MEMORY_ONLY_2, MEMORY_ONLY_SER, MEMORY_ONLY_SER_2, NONE, OFF_HEAP} +import java.util +import scala.collection.JavaConverters.mapAsScalaMapConverter import scala.util.control.NonFatal /** @@ -86,6 +93,15 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { Predicate.createInterpreted(e) } + override def createRelation(metaClient: HoodieTableMetaClient, + sqlContext: SQLContext, + schema: Schema, + globPaths: Array[Path], + parameters: util.Map[String, String]): BaseRelation = { + val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(schema) + DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) + } + /** * Converts instance of [[StorageLevel]] to a corresponding string */ From 7300c9eb17c30e11aaeb9cd768b15585536ab5f9 Mon Sep 17 00:00:00 2001 From: Rex An Date: Fri, 16 Sep 2022 17:24:20 +0800 Subject: [PATCH 12/18] Remove unused imports --- .../org/apache/spark/sql/hudi/SparkAdapter.scala | 3 +-- .../org/apache/spark/sql/adapter/Spark2Adapter.scala | 11 +++-------- .../apache/spark/sql/adapter/BaseSpark3Adapter.scala | 4 +--- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index a5ae82f146f80..f28d9835b7428 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -40,7 +40,6 @@ import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansU import org.apache.spark.storage.StorageLevel import java.util.Locale -import java.util.{Map => JMap} /** * Interface adapting discrepancies and incompatibilities between different Spark versions @@ -152,7 +151,7 @@ trait SparkAdapter extends Serializable { sqlContext: SQLContext, schema: Schema, globPaths: Array[Path], - parameters: JMap[String, String]): BaseRelation + parameters: java.util.Map[String, String]): BaseRelation /** * Create instance of [[HoodieFileScanRDD]] diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 2ed6b1c2022d4..755212948764a 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -20,12 +20,9 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema import org.apache.hadoop.fs.Path -import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL} -import org.apache.hudi.{AvroConversionUtils, BaseFileOnlyRelation, DefaultSource, EmptyRelation, HoodieBootstrapRelation, IncrementalRelation, MergeOnReadIncrementalRelation, MergeOnReadSnapshotRelation, Spark2HoodieFileScanRDD, Spark2RowSerDe} +import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark2HoodieFileScanRDD, Spark2RowSerDe} import org.apache.hudi.client.utils.SparkRowSerDe -import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.avro._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -37,7 +34,6 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, In import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile, Spark2ParsePartitionUtil, SparkParsePartitionUtil} -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser import org.apache.spark.sql.internal.SQLConf @@ -47,9 +43,8 @@ import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansU import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters.mapAsScalaMapConverter -import java.util.{Map => JMap} +import scala.collection.mutable.ArrayBuffer /** * Implementation of [[SparkAdapter]] for Spark 2.4.x @@ -136,7 +131,7 @@ class Spark2Adapter extends SparkAdapter { sqlContext: SQLContext, schema: Schema, globPaths: Array[Path], - parameters: JMap[String, String]): BaseRelation = { + parameters: java.util.Map[String, String]): BaseRelation = { val dataSchema = Option(schema).map(AvroConversionUtils.convertAvroSchemaToStructType).orNull DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index 51f6429b44e51..8f1c3cbce044c 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -22,7 +22,6 @@ import org.apache.hadoop.fs.Path import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark3RowSerDe} import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.exception.HoodieException import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -40,7 +39,6 @@ import org.apache.spark.sql.{HoodieCatalystPlansUtils, HoodieSpark3CatalystPlanU import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.{DISK_ONLY, DISK_ONLY_2, DISK_ONLY_3, MEMORY_AND_DISK, MEMORY_AND_DISK_2, MEMORY_AND_DISK_SER, MEMORY_AND_DISK_SER_2, MEMORY_ONLY, MEMORY_ONLY_2, MEMORY_ONLY_SER, MEMORY_ONLY_SER_2, NONE, OFF_HEAP} -import java.util import scala.collection.JavaConverters.mapAsScalaMapConverter import scala.util.control.NonFatal @@ -97,7 +95,7 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { sqlContext: SQLContext, schema: Schema, globPaths: Array[Path], - parameters: util.Map[String, String]): BaseRelation = { + parameters: java.util.Map[String, String]): BaseRelation = { val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(schema) DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) } From 090434e68beb265d90d15bd974a208b894c9b343 Mon Sep 17 00:00:00 2001 From: Rex An Date: Fri, 16 Sep 2022 17:59:39 +0800 Subject: [PATCH 13/18] Fix merge conflicts --- .../strategy/SparkSingleFileSortExecutionStrategy.java | 2 +- .../keygen/factory/HoodieSparkKeyGeneratorFactory.java | 9 +++++++-- .../scala/org/apache/spark/sql/hudi/SparkAdapter.scala | 2 +- .../spark/sql/catalyst/catalog/HoodieCatalogTable.scala | 2 +- .../org/apache/spark/sql/adapter/Spark2Adapter.scala | 5 ++++- 5 files changed, 14 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index ba14937601725..8885f727313e2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -89,7 +89,7 @@ public HoodieData performClusteringWithRecordsRDD(HoodieData fileGroupIdList, boolean shouldPreserveHoodieMetadata, - Map extraMetadata) { + Map extraMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java index 10141d74af447..5245c23f78018 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java @@ -76,11 +76,16 @@ public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx } } + public static String inferKeyGenClazz(TypedProperties props) { String partitionFields = props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), null); - if (partitionFields != null) { + String recordsKeyFields = props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()); + return inferKeyGenClazz(recordsKeyFields, partitionFields); + } + + public static String inferKeyGenClazz(String recordsKeyFields, String partitionFields) { + if (!StringUtils.isNullOrEmpty(partitionFields)) { int numPartFields = partitionFields.split(",").length; - String recordsKeyFields = props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()); int numRecordKeyFields = recordsKeyFields.split(",").length; if (numPartFields == 1 && numRecordKeyFields == 1) { return SimpleKeyGenerator.class.getName(); diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 7486bcf6b02bc..8a53b6b78e865 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, L import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SQLContext, SparkSession} +import org.apache.spark.sql.{HoodieCatalogUtils, HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SQLContext, SparkSession} import org.apache.spark.storage.StorageLevel import java.util.Locale diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index f1357723200d3..2beb907eca83c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -291,7 +291,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten val primaryKeys = table.properties.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName).getOrElse(SQL_KEY_TABLE_PRIMARY_KEY.defaultValue.get) val partitions = table.partitionColumnNames.mkString(",") extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = - DataSourceOptionsHelper.inferKeyGenClazz(primaryKeys, partitions) + HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(primaryKeys, partitions) } extraConfig.toMap } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 95a15587fd07e..2c690bbba48c8 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe -import org.apache.hudi.{Spark2HoodieFileScanRDD, Spark2RowSerDe} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark2HoodieFileScanRDD, Spark2RowSerDe} import org.apache.spark.sql.avro._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder @@ -34,6 +36,7 @@ import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql._ +import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ From 988e4874af3065d6879f9adc40c7483a84467f72 Mon Sep 17 00:00:00 2001 From: Rex An Date: Fri, 16 Sep 2022 18:24:08 +0800 Subject: [PATCH 14/18] Fix merging issue --- .../MultipleSparkJobExecutionStrategy.java | 6 ++++-- ...istentBucketClusteringExecutionStrategy.java | 14 ++++++++++++++ .../SparkSingleFileSortExecutionStrategy.java | 17 +++++++++-------- .../SparkSortAndSizeExecutionStrategy.java | 3 ++- .../factory/HoodieSparkKeyGeneratorFactory.java | 1 - 5 files changed, 29 insertions(+), 12 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 67ace9c4cf8d1..bd67edef915c8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -145,7 +145,8 @@ public abstract HoodieData performClusteringWithRecordsAsRow(final final Map strategyParams, final Schema schema, final List fileGroupIdList, - final boolean shouldPreserveHoodieMetadata); + final boolean shouldPreserveHoodieMetadata, + final Map extraMetadata); /** * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. @@ -245,7 +246,8 @@ private CompletableFuture> runClusteringForGroupAsyncAsR List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) .collect(Collectors.toList()); - return performClusteringWithRecordsAsRow(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, shouldPreserveHoodieMetadata); + return performClusteringWithRecordsAsRow(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, shouldPreserveHoodieMetadata, + clusteringGroup.getExtraMetadata()); }); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java index 9caea8508cfd8..91e46e3e4e73d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java @@ -35,6 +35,8 @@ import org.apache.avro.Schema; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; import java.util.List; import java.util.Map; @@ -53,6 +55,18 @@ public SparkConsistentBucketClusteringExecutionStrategy(HoodieTable table, Hoodi super(table, engineContext, writeConfig); } + @Override + public HoodieData performClusteringWithRecordsAsRow(Dataset inputRecords, + int numOutputGroups, + String instantTime, + Map strategyParams, + Schema schema, + List fileGroupIdList, + boolean shouldPreserveHoodieMetadata, + Map extraMetadata) { + throw new HoodieClusteringException("Not implement yet"); + } + @Override public HoodieData performClusteringWithRecordsRDD(HoodieData> inputRecords, int numOutputGroups, String instantTime, Map strategyParams, Schema schema, List fileGroupIdList, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index 8885f727313e2..f2ae9a922d811 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -64,7 +64,8 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in Map strategyParams, Schema schema, List fileGroupIdList, - boolean shouldPreserveHoodieMetadata) { + boolean shouldPreserveHoodieMetadata, + Map extraMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } @@ -83,13 +84,13 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in @Override public HoodieData performClusteringWithRecordsRDD(HoodieData> inputRecords, - int numOutputGroups, - String instantTime, - Map strategyParams, - Schema schema, - List fileGroupIdList, - boolean shouldPreserveHoodieMetadata, - Map extraMetadata) { + int numOutputGroups, + String instantTime, + Map strategyParams, + Schema schema, + List fileGroupIdList, + boolean shouldPreserveHoodieMetadata, + Map extraMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index fc09ec1012da1..35c8f288bc891 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -61,7 +61,8 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in String instantTime, Map strategyParams, Schema schema, List fileGroupIdList, - boolean shouldPreserveHoodieMetadata) { + boolean shouldPreserveHoodieMetadata, + Map extraMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() .withBulkInsertParallelism(numOutputGroups) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java index 5245c23f78018..837c409568e05 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java @@ -76,7 +76,6 @@ public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx } } - public static String inferKeyGenClazz(TypedProperties props) { String partitionFields = props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), null); String recordsKeyFields = props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()); From f2bb9e61707199197f30eef79e80db3e1241b3a0 Mon Sep 17 00:00:00 2001 From: Rex An Date: Fri, 16 Sep 2022 19:36:34 +0800 Subject: [PATCH 15/18] Fix test failure --- .../run/strategy/MultipleSparkJobExecutionStrategy.java | 4 +++- .../src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index bd67edef915c8..021980bb23156 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -394,8 +394,10 @@ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, paths = baseFilePaths; } + String readPathString = String.join(",", Arrays.stream(paths).map(Path::toString).toArray(String[]::new)); + params.put("hoodie.datasource.read.paths", readPathString); // Building HoodieFileIndex needs this param to decide query path - params.put("glob.paths", String.join(",", Arrays.stream(paths).map(Path::toString).toArray(String[]::new))); + params.put("glob.paths", readPathString); // Let Hudi relations to fetch the schema from the table itself BaseRelation relation = JavaSparkAdaptorSupport.sparkAdapter() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala index ba38dec812a30..01a28d056c643 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala @@ -160,6 +160,8 @@ class BaseFileOnlyRelation(sqlContext: SQLContext, fileFormat = fileFormat, optParams)(sparkSession) } else { + val readPathsStr = optParams.get(DataSourceReadOptions.READ_PATHS.key) + val extraReadPaths = readPathsStr.map(p => p.split(",").toSeq).getOrElse(Seq()) // NOTE: Spark is able to infer partitioning values from partition path only when Hive-style partitioning // scheme is used. Therefore, we fallback to reading the table as non-partitioned (specifying // partitionColumns = Seq.empty) whenever Hive-style partitioning is not involved @@ -171,7 +173,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext, DataSource.apply( sparkSession = sparkSession, - paths = globPaths.map(_.toString), + paths = extraReadPaths, // Here we should specify the schema to the latest commit schema since // the table schema evolution. userSpecifiedSchema = userSchema.orElse(Some(tableStructSchema)), From 1587f472f18d7b524971637abe64d171c9799818 Mon Sep 17 00:00:00 2001 From: Rex An Date: Fri, 16 Sep 2022 23:14:49 +0800 Subject: [PATCH 16/18] Fix NPE for spark3 --- .../scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index 0fe2a61f31b41..c9126fdf0d8f8 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -98,7 +98,7 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { schema: Schema, globPaths: Array[Path], parameters: java.util.Map[String, String]): BaseRelation = { - val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val dataSchema = Option(schema).map(AvroConversionUtils.convertAvroSchemaToStructType).orNull DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) } From 20f64af242ac3e6df5d1555edf0766e7dcdd698a Mon Sep 17 00:00:00 2001 From: Rex An Date: Sun, 18 Sep 2022 13:23:23 +0800 Subject: [PATCH 17/18] Remove unused changes --- .../HoodieSparkKeyGeneratorFactory.java | 21 ----------------- .../hudi/HoodieDatasetBulkInsertHelper.scala | 11 +++------ .../org/apache/hudi/DataSourceOptions.scala | 23 +++++++++++++++++-- .../catalyst/catalog/HoodieCatalogTable.scala | 2 +- .../sql/hudi/command/SqlKeyGenerator.scala | 2 +- 5 files changed, 26 insertions(+), 33 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java index 837c409568e05..165b27d6ce283 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java @@ -30,7 +30,6 @@ import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.keygen.TimestampBasedKeyGenerator; -import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; import org.slf4j.Logger; @@ -76,26 +75,6 @@ public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx } } - public static String inferKeyGenClazz(TypedProperties props) { - String partitionFields = props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), null); - String recordsKeyFields = props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()); - return inferKeyGenClazz(recordsKeyFields, partitionFields); - } - - public static String inferKeyGenClazz(String recordsKeyFields, String partitionFields) { - if (!StringUtils.isNullOrEmpty(partitionFields)) { - int numPartFields = partitionFields.split(",").length; - int numRecordKeyFields = recordsKeyFields.split(",").length; - if (numPartFields == 1 && numRecordKeyFields == 1) { - return SimpleKeyGenerator.class.getName(); - } else { - return ComplexKeyGenerator.class.getName(); - } - } else { - return NonpartitionedKeyGenerator.class.getName(); - } - } - public static String getKeyGeneratorClassName(TypedProperties props) { String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), null); diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 15f5dd019cced..296abaf4f5e27 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload} import org.apache.hudi.common.util.ReflectionUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.index.SparkHoodieIndexFactory -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{BuiltinKeyGenerator, SparkKeyGeneratorInterface} import org.apache.hudi.table.{BulkInsertPartitioner, HoodieTable} import org.apache.hudi.table.action.commit.BulkInsertDataInternalWriterHelper @@ -59,7 +58,8 @@ object HoodieDatasetBulkInsertHelper extends Logging { val populateMetaFields = config.populateMetaFields() val schema = df.schema - val keyGeneratorClassName = getKeyGenClassName(config) + val keyGeneratorClassName = config.getStringOrThrow(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME, + "Key-generator class name is required") val prependedRdd: RDD[InternalRow] = df.queryExecution.toRdd.mapPartitions { iter => @@ -154,11 +154,6 @@ object HoodieDatasetBulkInsertHelper extends Logging { table.getContext.parallelize(writeStatuses.toList.asJava) } - private def getKeyGenClassName(config: HoodieWriteConfig): String = { - Option(config.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key())) - .getOrElse(HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(config.getProps)) - } - private def dedupeRows(rdd: RDD[InternalRow], schema: StructType, preCombineFieldRef: String, isGlobalIndex: Boolean): RDD[InternalRow] = { val recordKeyMetaFieldOrd = schema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD) val partitionPathMetaFieldOrd = schema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD) @@ -203,7 +198,7 @@ object HoodieDatasetBulkInsertHelper extends Logging { } private def getPartitionPathFields(config: HoodieWriteConfig): Seq[String] = { - val keyGeneratorClassName = getKeyGenClassName(config) + val keyGeneratorClassName = config.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME) val keyGenerator = ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps)).asInstanceOf[BuiltinKeyGenerator] keyGenerator.getPartitionPathFields.asScala } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index c381c265c5d11..e8ffb09ff9100 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -28,7 +28,6 @@ import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.keygen.constant.KeyGeneratorOptions -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.hudi.sync.common.util.ConfigUtils @@ -327,7 +326,7 @@ object DataSourceWriteOptions { * Key generator class, that implements will extract the key out of incoming record. */ val keyGeneraterInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { - Option.of(HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(p.getProps)) + Option.of(DataSourceOptionsHelper.inferKeyGenClazz(p.getProps)) }) val KEYGENERATOR_CLASS_NAME: ConfigProperty[String] = ConfigProperty @@ -786,6 +785,26 @@ object DataSourceOptionsHelper { ) ++ translateConfigurations(paramsWithGlobalProps) } + def inferKeyGenClazz(props: TypedProperties): String = { + val partitionFields = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), null) + val recordsKeyFields = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD.key(), DataSourceWriteOptions.RECORDKEY_FIELD.defaultValue()) + inferKeyGenClazz(recordsKeyFields, partitionFields) + } + + def inferKeyGenClazz(recordsKeyFields: String, partitionFields: String): String = { + if (!StringUtils.isNullOrEmpty(partitionFields)) { + val numPartFields = partitionFields.split(",").length + val numRecordKeyFields = recordsKeyFields.split(",").length + if (numPartFields == 1 && numRecordKeyFields == 1) { + classOf[SimpleKeyGenerator].getName + } else { + classOf[ComplexKeyGenerator].getName + } + } else { + classOf[NonpartitionedKeyGenerator].getName + } + } + implicit def scalaFunctionToJavaFunction[From, To](function: (From) => To): JavaFunction[From, To] = { new JavaFunction[From, To] { override def apply (input: From): To = function (input) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index 2beb907eca83c..f1357723200d3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -291,7 +291,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten val primaryKeys = table.properties.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName).getOrElse(SQL_KEY_TABLE_PRIMARY_KEY.defaultValue.get) val partitions = table.partitionColumnNames.mkString(",") extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = - HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(primaryKeys, partitions) + DataSourceOptionsHelper.inferKeyGenClazz(primaryKeys, partitions) } extraConfig.toMap } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala index ca160ff7d59b1..798ed84b0939c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala @@ -136,7 +136,7 @@ object SqlKeyGenerator { if (beforeKeyGenClassName != null && beforeKeyGenClassName.nonEmpty) { HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(beforeKeyGenClassName) } else { - HoodieSparkKeyGeneratorFactory.inferKeyGenClazz(props) + DataSourceOptionsHelper.inferKeyGenClazz(props) } } } From 2ff0b70e69fcff7cd061a2512dc983ac92a3c87c Mon Sep 17 00:00:00 2001 From: Rex An Date: Tue, 20 Sep 2022 11:14:05 +0800 Subject: [PATCH 18/18] Address feedbacks --- .../apache/hudi/JavaSparkAdaptorSupport.java | 59 ------------------- .../MultipleSparkJobExecutionStrategy.java | 6 +- .../org/apache/hudi/SparkAdapterSupport.scala | 19 +++++- .../apache/spark/sql/hudi/SparkAdapter.scala | 4 +- .../scala/org/apache/hudi/DefaultSource.scala | 15 ++--- ...HoodieSparkMergeOnReadTableClustering.java | 38 ++++++------ .../functional/TestLayoutOptimization.scala | 51 ++++++++-------- .../spark/sql/adapter/Spark2Adapter.scala | 6 +- .../spark/sql/adapter/BaseSpark3Adapter.scala | 6 +- 9 files changed, 81 insertions(+), 123 deletions(-) delete mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java deleted file mode 100644 index 1bfca6ac3b323..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/JavaSparkAdaptorSupport.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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; - -import org.apache.hudi.exception.HoodieException; -import org.apache.spark.sql.hudi.SparkAdapter; - -/** - * Java implementation to provide SparkAdapter when we need to adapt - * the difference between spark2 and spark3. - */ -public class JavaSparkAdaptorSupport { - - private JavaSparkAdaptorSupport() {} - - private static class AdapterSupport { - - private static final SparkAdapter ADAPTER = new AdapterSupport().sparkAdapter(); - - private SparkAdapter sparkAdapter() { - String adapterClass; - if (HoodieSparkUtils.isSpark3_3()) { - adapterClass = "org.apache.spark.sql.adapter.Spark3_3Adapter"; - } else if (HoodieSparkUtils.isSpark3_2()) { - adapterClass = "org.apache.spark.sql.adapter.Spark3_2Adapter"; - } else if (HoodieSparkUtils.isSpark3_0() || HoodieSparkUtils.isSpark3_1()) { - adapterClass = "org.apache.spark.sql.adapter.Spark3_1Adapter"; - } else { - adapterClass = "org.apache.spark.sql.adapter.Spark2Adapter"; - } - try { - return (SparkAdapter) this.getClass().getClassLoader().loadClass(adapterClass) - .newInstance(); - } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { - throw new HoodieException("Cannot instantiate SparkAdaptor", e); - } - } - } - - public static SparkAdapter sparkAdapter() { - return AdapterSupport.ADAPTER; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 021980bb23156..0c52a7cc49b7e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -18,7 +18,7 @@ package org.apache.hudi.client.clustering.run.strategy; -import org.apache.hudi.JavaSparkAdaptorSupport; +import org.apache.hudi.SparkAdapterSupport$; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -400,8 +400,8 @@ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, params.put("glob.paths", readPathString); // Let Hudi relations to fetch the schema from the table itself - BaseRelation relation = JavaSparkAdaptorSupport.sparkAdapter() - .createRelation(getHoodieTable().getMetaClient(), sqlContext, null, paths, params); + BaseRelation relation = SparkAdapterSupport$.MODULE$.sparkAdapter() + .createRelation(sqlContext, getHoodieTable().getMetaClient(), null, paths, params); return sqlContext.baseRelationToDataFrame(relation); } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala index b0eac3d956fbc..9fe67f9918d01 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala @@ -26,6 +26,23 @@ import org.apache.spark.sql.hudi.SparkAdapter */ trait SparkAdapterSupport { - lazy val sparkAdapter: SparkAdapter = JavaSparkAdaptorSupport.sparkAdapter() + lazy val sparkAdapter: SparkAdapter = SparkAdapterSupport.sparkAdapter } + +object SparkAdapterSupport { + + lazy val sparkAdapter: SparkAdapter = { + val adapterClass = if (HoodieSparkUtils.isSpark3_3) { + "org.apache.spark.sql.adapter.Spark3_3Adapter" + } else if (HoodieSparkUtils.isSpark3_2) { + "org.apache.spark.sql.adapter.Spark3_2Adapter" + } else if (HoodieSparkUtils.isSpark3_0 || HoodieSparkUtils.isSpark3_1) { + "org.apache.spark.sql.adapter.Spark3_1Adapter" + } else { + "org.apache.spark.sql.adapter.Spark2Adapter" + } + getClass.getClassLoader.loadClass(adapterClass) + .newInstance().asInstanceOf[SparkAdapter] + } +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 8a53b6b78e865..6f9616b669c47 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -153,8 +153,8 @@ trait SparkAdapter extends Serializable { /** * Create Hoodie relation based on globPaths, otherwise use tablePath if it's empty */ - def createRelation(metaClient: HoodieTableMetaClient, - sqlContext: SQLContext, + def createRelation(sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, schema: Schema, globPaths: Array[Path], parameters: java.util.Map[String, String]): BaseRelation diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 26ca565339d6d..c7608c98291be 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -107,12 +107,8 @@ class DefaultSource extends RelationProvider log.info("Obtained hudi table path: " + tablePath) val metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(tablePath).build() - val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent - val tableType = metaClient.getTableType - val queryType = parameters(QUERY_TYPE.key) - log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType") - DefaultSource.createRelation(metaClient, sqlContext, schema, globPaths, parameters) + DefaultSource.createRelation(sqlContext, metaClient, schema, globPaths, parameters) } def getValidCommits(metaClient: HoodieTableMetaClient): String = { @@ -199,14 +195,19 @@ class DefaultSource extends RelationProvider object DefaultSource { - def createRelation(metaClient: HoodieTableMetaClient, - sqlContext: SQLContext, + private val log = LogManager.getLogger(classOf[DefaultSource]) + + def createRelation(sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, schema: StructType, globPaths: Seq[Path], parameters: Map[String, String]): BaseRelation = { val tableType = metaClient.getTableType val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent val queryType = parameters(QUERY_TYPE.key) + + log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType") + // NOTE: In cases when Hive Metastore is used as catalog and the table is partitioned, schema in the HMS might contain // Hive-specific partitioning columns created specifically for HMS to handle partitioning appropriately. In that // case we opt in to not be providing catalog's schema, and instead force Hudi relations to fetch the schema diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java index 06114c2bf277d..a343ab3c5ccee 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -47,7 +47,6 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Stream; @@ -60,26 +59,25 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTestHarness { private static Stream testClustering() { + // enableClusteringAsRow, doUpdates, populateMetaFields, preserveCommitMetadata return Stream.of( - Arrays.asList(true, true, true), - Arrays.asList(true, true, false), - Arrays.asList(true, false, true), - Arrays.asList(true, false, false), - Arrays.asList(false, true, true), - Arrays.asList(false, true, false), - Arrays.asList(false, false, true), - Arrays.asList(false, false, false)) - .flatMap(arguments -> { - ArrayList enableRowClusteringArgs = new ArrayList<>(); - enableRowClusteringArgs.add(true); - enableRowClusteringArgs.addAll(arguments); - ArrayList disableRowClusteringArgs = new ArrayList<>(); - disableRowClusteringArgs.add(false); - disableRowClusteringArgs.addAll(arguments); - return Stream.of( - Arguments.of(enableRowClusteringArgs.toArray(new Boolean[0])), - Arguments.of(disableRowClusteringArgs.toArray(new Boolean[0]))); - }); + Arguments.of(true, true, true, true), + Arguments.of(true, true, true, false), + Arguments.of(true, true, false, true), + Arguments.of(true, true, false, false), + Arguments.of(true, false, true, true), + Arguments.of(true, false, true, false), + Arguments.of(true, false, false, true), + Arguments.of(true, false, false, false), + Arguments.of(false, true, true, true), + Arguments.of(false, true, true, false), + Arguments.of(false, true, false, true), + Arguments.of(false, true, false, false), + Arguments.of(false, false, true, true), + Arguments.of(false, false, true, false), + Arguments.of(false, false, false, true), + Arguments.of(false, false, false, false) + ); } @ParameterizedTest diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala index 265a1588ce781..7d54959d2ea4c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala @@ -31,10 +31,9 @@ import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag} import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments.arguments import org.junit.jupiter.params.provider.{Arguments, MethodSource} -import java.util -import java.util.{function, stream} import scala.collection.JavaConversions._ @Tag("functional") @@ -85,8 +84,8 @@ class TestLayoutOptimization extends HoodieClientTestBase { @ParameterizedTest @MethodSource(Array("testLayoutOptimizationParameters")) - def testLayoutOptimizationFunctional(clusteringAsRow: String, - tableType: String, + def testLayoutOptimizationFunctional(tableType: String, + clusteringAsRow: String, layoutOptimizationStrategy: String, spatialCurveCompositionStrategy: String): Unit = { val curveCompositionStrategy = @@ -167,27 +166,29 @@ class TestLayoutOptimization extends HoodieClientTestBase { object TestLayoutOptimization { def testLayoutOptimizationParameters(): java.util.stream.Stream[Arguments] = { + // TableType, enableClusteringAsRow, layoutOptimizationStrategy, spatialCurveCompositionStrategy java.util.stream.Stream.of( - Seq("COPY_ON_WRITE", "linear", null), - Seq("COPY_ON_WRITE", "z-order", "direct"), - Seq("COPY_ON_WRITE", "z-order", "sample"), - Seq("COPY_ON_WRITE", "hilbert", "direct"), - Seq("COPY_ON_WRITE", "hilbert", "sample"), - Seq("MERGE_ON_READ", "linear", null), - Seq("MERGE_ON_READ", "z-order", "direct"), - Seq("MERGE_ON_READ", "z-order", "sample"), - Seq("MERGE_ON_READ", "hilbert", "direct"), - Seq("MERGE_ON_READ", "hilbert", "sample") - ).flatMap(new function.Function[Seq[String], util.stream.Stream[Arguments]] { - override def apply(args: Seq[String]): stream.Stream[Arguments] = { - val enableRowClusteringArgs = "true" +: args - val disableRowClusteringArgs = "false" +: args - - java.util.stream.Stream.of( - Arguments.of(enableRowClusteringArgs.toArray:_*), - Arguments.of(disableRowClusteringArgs.toArray:_*) - ) - } - }) + arguments("COPY_ON_WRITE", "true", "linear", null), + arguments("COPY_ON_WRITE", "true", "z-order", "direct"), + arguments("COPY_ON_WRITE", "true", "z-order", "sample"), + arguments("COPY_ON_WRITE", "true", "hilbert", "direct"), + arguments("COPY_ON_WRITE", "true", "hilbert", "sample"), + arguments("COPY_ON_WRITE", "false", "linear", null), + arguments("COPY_ON_WRITE", "false", "z-order", "direct"), + arguments("COPY_ON_WRITE", "false", "z-order", "sample"), + arguments("COPY_ON_WRITE", "false", "hilbert", "direct"), + arguments("COPY_ON_WRITE", "false", "hilbert", "sample"), + + arguments("MERGE_ON_READ", "true", "linear", null), + arguments("MERGE_ON_READ", "true", "z-order", "direct"), + arguments("MERGE_ON_READ", "true", "z-order", "sample"), + arguments("MERGE_ON_READ", "true", "hilbert", "direct"), + arguments("MERGE_ON_READ", "true", "hilbert", "sample"), + arguments("MERGE_ON_READ", "false", "linear", null), + arguments("MERGE_ON_READ", "false", "z-order", "direct"), + arguments("MERGE_ON_READ", "false", "z-order", "sample"), + arguments("MERGE_ON_READ", "false", "hilbert", "direct"), + arguments("MERGE_ON_READ", "false", "hilbert", "sample") + ) } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 2c690bbba48c8..d4ca8bc402015 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -129,13 +129,13 @@ class Spark2Adapter extends SparkAdapter { InterpretedPredicate.create(e) } - override def createRelation(metaClient: HoodieTableMetaClient, - sqlContext: SQLContext, + override def createRelation(sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, schema: Schema, globPaths: Array[Path], parameters: java.util.Map[String, String]): BaseRelation = { val dataSchema = Option(schema).map(AvroConversionUtils.convertAvroSchemaToStructType).orNull - DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) + DefaultSource.createRelation(sqlContext, metaClient, dataSchema, globPaths, parameters.asScala.toMap) } override def createHoodieFileScanRDD(sparkSession: SparkSession, diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index c9126fdf0d8f8..d0fb7ce6e8651 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -93,13 +93,13 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { Predicate.createInterpreted(e) } - override def createRelation(metaClient: HoodieTableMetaClient, - sqlContext: SQLContext, + override def createRelation(sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, schema: Schema, globPaths: Array[Path], parameters: java.util.Map[String, String]): BaseRelation = { val dataSchema = Option(schema).map(AvroConversionUtils.convertAvroSchemaToStructType).orNull - DefaultSource.createRelation(metaClient, sqlContext, dataSchema, globPaths, parameters.asScala.toMap) + DefaultSource.createRelation(sqlContext, metaClient, dataSchema, globPaths, parameters.asScala.toMap) } /**