diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index f701e4036bdd0..ac1c349374172 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -99,7 +99,7 @@ public abstract class HoodieTable implem protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; - protected final HoodieIndex index; + protected final HoodieIndex index; private SerializableConfiguration hadoopConfiguration; protected final TaskContextSupplier taskContextSupplier; private final HoodieTableMetadata metadata; @@ -123,7 +123,7 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo this.taskContextSupplier = context.getTaskContextSupplier(); } - protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); + protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { @@ -345,7 +345,7 @@ public HoodieActiveTimeline getActiveTimeline() { /** * Return the index. */ - public HoodieIndex getIndex() { + public HoodieIndex getIndex() { return index; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java index 5ef204f9706db..dd43ca3b9cf59 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java @@ -46,6 +46,29 @@ public class HoodieWriteMetadata { public HoodieWriteMetadata() { } + public HoodieWriteMetadata clone(T transformedWriteStatuses) { + HoodieWriteMetadata newMetadataInstance = new HoodieWriteMetadata<>(); + newMetadataInstance.setWriteStatuses(transformedWriteStatuses); + if (indexLookupDuration.isPresent()) { + newMetadataInstance.setIndexLookupDuration(indexLookupDuration.get()); + } + newMetadataInstance.setCommitted(isCommitted); + newMetadataInstance.setCommitMetadata(commitMetadata); + if (writeStats.isPresent()) { + newMetadataInstance.setWriteStats(writeStats.get()); + } + if (indexUpdateDuration.isPresent()) { + newMetadataInstance.setIndexUpdateDuration(indexUpdateDuration.get()); + } + if (finalizeDuration.isPresent()) { + newMetadataInstance.setFinalizeDuration(finalizeDuration.get()); + } + if (partitionToReplaceFileIds.isPresent()) { + newMetadataInstance.setPartitionToReplaceFileIds(partitionToReplaceFileIds.get()); + } + return newMetadataInstance; + } + public O getWriteStatuses() { return writeStatuses; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapHelper.java new file mode 100644 index 0000000000000..61d583e4844e7 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapHelper.java @@ -0,0 +1,39 @@ +/* + * 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.bootstrap; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.BaseCommitHelper; + +import java.io.Serializable; +import java.util.Map; + +public abstract class BaseBootstrapHelper> implements Serializable { + public abstract HoodieBootstrapWriteMetadata> execute( + HoodieEngineContext context, HoodieTable table, HoodieWriteConfig config, + Option> extraMetadata, + BaseCommitHelper commitHelper); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java new file mode 100644 index 0000000000000..ccf4372fc3f98 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java @@ -0,0 +1,54 @@ +/* + * 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.bootstrap; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.BaseCommitActionExecutor; +import org.apache.hudi.table.action.commit.BaseCommitHelper; + +import java.util.Map; + +public class BootstrapCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final BaseBootstrapHelper bootstrapHelper; + + public BootstrapCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + Option> extraMetadata, + BaseCommitHelper commitHelper, BaseBootstrapHelper bootstrapHelper) { + super(context, config, table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + WriteOperationType.BOOTSTRAP, extraMetadata, commitHelper); + this.bootstrapHelper = bootstrapHelper; + } + + @Override + public HoodieBootstrapWriteMetadata> execute() { + return bootstrapHelper.execute(context, table, config, extraMetadata, commitHelper); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanHelper.java new file mode 100644 index 0000000000000..5e277b390801e --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanHelper.java @@ -0,0 +1,32 @@ +/* + * 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.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +public abstract class BaseClusteringPlanHelper> { + public abstract Option createClusteringPlan( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java similarity index 75% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java index 97407e3464d79..e21934f7e544f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java @@ -20,7 +20,10 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -28,6 +31,7 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; @@ -36,24 +40,24 @@ import java.util.Collections; import java.util.Map; -public abstract class BaseClusteringPlanActionExecutor extends BaseActionExecutor> { +public class ClusteringPlanActionExecutor> + extends BaseActionExecutor>, HoodieData, HoodieData, Option> { + private final BaseClusteringPlanHelper clusteringPlanHelper; private final Option> extraMetadata; - public BaseClusteringPlanActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - Option> extraMetadata) { + public ClusteringPlanActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, Option> extraMetadata, + BaseClusteringPlanHelper clusteringPlanHelper) { super(context, config, table, instantTime); this.extraMetadata = extraMetadata; + this.clusteringPlanHelper = clusteringPlanHelper; } - protected abstract Option createClusteringPlan(); - @Override public Option execute() { - Option planOption = createClusteringPlan(); + Option planOption = clusteringPlanHelper.createClusteringPlan(context, config, table); if (planOption.isPresent()) { HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java similarity index 57% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java index 850f3e0761851..c727bd2e02482 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java @@ -18,31 +18,33 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public abstract class AbstractBulkInsertHelper { +public abstract class BaseBulkInsertHelper> { /** * Mark instant as inflight, write input records, update index and return result. */ - public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instantTime, - HoodieTable table, HoodieWriteConfig config, - BaseCommitActionExecutor executor, boolean performDedupe, - Option> userDefinedBulkInsertPartitioner); + public abstract HoodieWriteMetadata> bulkInsert( + HoodieData> inputRecords, String instantTime, HoodieTable table, + HoodieWriteConfig config, boolean performDedupe, + Option>>> userDefinedBulkInsertPartitioner, + BaseCommitHelper commitHelper); /** * Only write input records. Does not change timeline/index. Return information about new files created. */ - public abstract O bulkInsert(I inputRecords, String instantTime, - HoodieTable table, HoodieWriteConfig config, - boolean performDedupe, - Option> userDefinedBulkInsertPartitioner, - boolean addMetadataFields, - int parallelism, - boolean preserveMetadata); + public abstract HoodieData bulkInsert( + HoodieData> inputRecords, String instantTime, HoodieTable table, + HoodieWriteConfig config, boolean performDedupe, + Option>>> userDefinedBulkInsertPartitioner, + boolean addMetadataFields, int parallelism, boolean preserveMetadata); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseClusteringHelper.java similarity index 63% rename from hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseClusteringHelper.java index 0b4a654074408..573adb6ad7bf6 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseClusteringHelper.java @@ -17,19 +17,22 @@ * under the License. */ -package org.apache.hudi.table.action.deltacommit; +package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.commit.BaseJavaCommitActionExecutor; +import org.apache.hudi.table.action.HoodieWriteMetadata; -public abstract class BaseJavaDeltaCommitActionExecutor> extends BaseJavaCommitActionExecutor { +import java.util.Map; - public BaseJavaDeltaCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType) { - super(context, config, table, instantTime, operationType); - } +public abstract class BaseClusteringHelper> { + public abstract HoodieWriteMetadata> execute( + HoodieEngineContext context, String instantTime, HoodieTable table, HoodieWriteConfig config, + Option> extraMetadata, + BaseCommitHelper commitHelper); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 4b519ed92b4aa..46c6f2e820660 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -19,180 +19,83 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.transaction.TransactionManager; -import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCommitException; -import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.WorkloadProfile; -import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.apache.hadoop.conf.Configuration; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; import java.util.Iterator; import java.util.List; import java.util.Map; -public abstract class BaseCommitActionExecutor - extends BaseActionExecutor { - - private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class); +public abstract class BaseCommitActionExecutor, R> + extends BaseActionExecutor>, HoodieData, HoodieData, R> { + protected final BaseCommitHelper commitHelper; protected final Option> extraMetadata; protected final WriteOperationType operationType; protected final TaskContextSupplier taskContextSupplier; - protected final TransactionManager txnManager; - protected Option>> lastCompletedTxn; - public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, - HoodieTable table, String instantTime, WriteOperationType operationType, - Option> extraMetadata) { + public BaseCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + String instantTime, WriteOperationType operationType, + BaseCommitHelper commitHelper) { + this(context, config, table, instantTime, operationType, Option.empty(), commitHelper); + } + + public BaseCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata, + BaseCommitHelper commitHelper) { super(context, config, table, instantTime); + this.commitHelper = commitHelper; this.operationType = operationType; this.extraMetadata = extraMetadata; this.taskContextSupplier = context.getTaskContextSupplier(); - // TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link AbstractHoodieWriteClient}. - this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); - this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); + commitHelper.init(config); } - public abstract HoodieWriteMetadata execute(I inputRecords); - - /** - * Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing - * rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks - * are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO : - * Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata - */ - void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String instantTime) - throws HoodieCommitException { - try { - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - profile.getPartitionPaths().forEach(path -> { - WorkloadStat partitionStat = profile.getWorkloadStat(path); - HoodieWriteStat insertStat = new HoodieWriteStat(); - insertStat.setNumInserts(partitionStat.getNumInserts()); - insertStat.setFileId(""); - insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); - metadata.addWriteStat(path, insertStat); - - partitionStat.getUpdateLocationToCount().forEach((key, value) -> { - HoodieWriteStat writeStat = new HoodieWriteStat(); - writeStat.setFileId(key); - // TODO : Write baseCommitTime is possible here ? - writeStat.setPrevCommit(value.getKey()); - writeStat.setNumUpdateWrites(value.getValue()); - metadata.addWriteStat(path, writeStat); - }); - }); - metadata.setOperationType(operationType); - - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = getCommitActionType(); - HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); - activeTimeline.transitionRequestedToInflight(requested, - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)), - config.shouldAllowMultiWriteOnSameInstant()); - } catch (IOException io) { - throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); - } + public HoodieWriteMetadata> execute( + HoodieData> inputRecords) { + return commitHelper.execute(inputRecords); } - protected String getCommitActionType() { - return table.getMetaClient().getCommitActionType(); + public Configuration getHadoopConf() { + return hadoopConf; } - - /** - * Check if any validators are configured and run those validations. If any of the validations fail, throws HoodieValidationException. - */ - protected void runPrecommitValidators(HoodieWriteMetadata writeMetadata) { - if (StringUtils.isNullOrEmpty(config.getPreCommitValidators())) { - return; - } - throw new HoodieIOException("Precommit validation not implemented for all engines yet"); - } - - protected void commitOnAutoCommit(HoodieWriteMetadata result) { - // validate commit action before committing result - runPrecommitValidators(result); - if (config.shouldAutoCommit()) { - LOG.info("Auto commit enabled: Committing " + instantTime); - autoCommit(extraMetadata, result); - } else { - LOG.info("Auto commit disabled for " + instantTime); - } - } - - protected void autoCommit(Option> extraMetadata, HoodieWriteMetadata result) { - this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime)), - lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); - try { - TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), - result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner()); - commit(extraMetadata, result); - } finally { - this.txnManager.endTransaction(); - } - } - - protected abstract void commit(Option> extraMetadata, HoodieWriteMetadata result); - - /** - * Finalize Write operation. - * @param instantTime Instant Time - * @param stats Hoodie Write Stat - */ - protected void finalizeWrite(String instantTime, List stats, HoodieWriteMetadata result) { - try { - Instant start = Instant.now(); - table.finalizeWrite(context, instantTime, stats); - result.setFinalizeDuration(Duration.between(start, Instant.now())); - } catch (HoodieIOException ioe) { - throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe); - } + protected String getCommitActionType() { + return table.getMetaClient().getCommitActionType(); } - protected void syncTableMetadata() { - // No Op + protected void commit( + Option> extraMetadata, + HoodieWriteMetadata> result) { + commitHelper.commit(extraMetadata, result); } - /** - * By default, return the writer schema in Write Config for storing in commit. - */ - protected String getSchemaToStoreInCommit() { - return config.getSchema(); + protected Iterator> handleInsert( + String idPfx, Iterator> recordItr) throws Exception { + return commitHelper.handleInsert(idPfx, recordItr); } - protected boolean isWorkloadProfileNeeded() { - return true; + protected Iterator> handleUpdate( + String partitionPath, String fileId, + Iterator> recordItr) throws IOException { + return commitHelper.handleUpdate(partitionPath, fileId, recordItr); } - - protected abstract Iterator> handleInsert(String idPfx, - Iterator> recordItr) throws Exception; - - protected abstract Iterator> handleUpdate(String partitionPath, String fileId, - Iterator> recordItr) throws IOException; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitHelper.java new file mode 100644 index 0000000000000..bd9eaeddea07f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitHelper.java @@ -0,0 +1,295 @@ +/* + * 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.client.transaction.TransactionManager; +import org.apache.hudi.client.utils.TransactionUtils; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public abstract class BaseCommitHelper> implements Serializable { + + private static final Logger LOG = LogManager.getLogger(BaseCommitHelper.class); + + protected final transient HoodieEngineContext context; + protected final HoodieWriteConfig config; + protected final HoodieTable>, HoodieData, HoodieData> table; + protected final String instantTime; + protected final Option> extraMetadata; + protected final WriteOperationType operationType; + protected final TransactionManager txnManager; + protected final TaskContextSupplier taskContextSupplier; + protected Option>> lastCompletedTxn; + + public BaseCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + String instantTime, WriteOperationType operationType) { + this(context, config, table, instantTime, operationType, Option.empty()); + } + + public BaseCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + this.context = context; + this.config = config; + this.table = table; + this.instantTime = instantTime; + this.operationType = operationType; + this.extraMetadata = extraMetadata; + // TODO : Remove this once we refactor and move out autoCommit method from here, + // since the TxnManager is held in {@link AbstractHoodieWriteClient}. + this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); + this.taskContextSupplier = context.getTaskContextSupplier(); + this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); + } + + public void commit( + Option> extraMetadata, + HoodieWriteMetadata> result) { + context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect"); + commit(extraMetadata, result, context.map(result.getWriteStatuses(), WriteStatus::getStat)); + } + + public Iterator> handleUpdate( + String partitionPath, String fileId, + Iterator> recordItr) throws IOException { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + LOG.info("Empty partition with fileId => " + fileId); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } + // these are updates + + HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); + return handleUpdateInternal(upsertHandle, fileId); + } + + /** + * By default, return the writer schema in Write Config for storing in commit. + */ + public String getSchemaToStoreInCommit() { + return config.getSchema(); + } + + public abstract void init(HoodieWriteConfig config); + + public abstract HoodieWriteMetadata> execute( + HoodieData> inputRecords); + + public abstract Iterator> handleInsert( + String idPfx, Iterator> recordItr) throws Exception; + + protected abstract void syncTableMetadata(); + + protected abstract HoodieMergeHandle getUpdateHandle( + String partitionPath, String fileId, Iterator> recordItr); + + protected abstract BaseMergeHelper getMergeHelper(); + + /** + * Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing + * rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks + * are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO : + * Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata + */ + void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String instantTime) + throws HoodieCommitException { + try { + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + profile.getPartitionPaths().forEach(path -> { + WorkloadStat partitionStat = profile.getWorkloadStat(path); + HoodieWriteStat insertStat = new HoodieWriteStat(); + insertStat.setNumInserts(partitionStat.getNumInserts()); + insertStat.setFileId(""); + insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); + metadata.addWriteStat(path, insertStat); + + partitionStat.getUpdateLocationToCount().forEach((key, value) -> { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(key); + // TODO : Write baseCommitTime is possible here ? + writeStat.setPrevCommit(value.getKey()); + writeStat.setNumUpdateWrites(value.getValue()); + metadata.addWriteStat(path, writeStat); + }); + }); + metadata.setOperationType(operationType); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + String commitActionType = getCommitActionType(); + HoodieInstant requested = + new HoodieInstant(HoodieInstant.State.REQUESTED, commitActionType, instantTime); + activeTimeline.transitionRequestedToInflight(requested, + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)), + config.shouldAllowMultiWriteOnSameInstant()); + } catch (IOException io) { + throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); + } + } + + protected String getCommitActionType() { + return table.getMetaClient().getCommitActionType(); + } + + protected boolean isWorkloadProfileNeeded() { + return true; + } + + /** + * Check if any validators are configured and run those validations. + * If any of the validations fail, throws HoodieValidationException. + */ + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + if (StringUtils.isNullOrEmpty(config.getPreCommitValidators())) { + return; + } + throw new HoodieIOException("Precommit validation not implemented for all engines yet"); + } + + protected void commit( + Option> extraMetadata, + HoodieWriteMetadata> result, List writeStats) { + String actionType = getCommitActionType(); + LOG.info("Committing " + instantTime + ", action Type " + actionType); + result.setCommitted(true); + result.setWriteStats(writeStats); + // Finalize write + finalizeWrite(instantTime, writeStats, result); + syncTableMetadata(); + try { + LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), + extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + LOG.info("Committed " + instantTime); + result.setCommitMetadata(Option.of(metadata)); + } catch (IOException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, + e); + } + } + + public void commitOnAutoCommit(HoodieWriteMetadata> result) { + // validate commit action before committing result + runPrecommitValidators(result); + if (config.shouldAutoCommit()) { + LOG.info("Auto commit enabled: Committing " + instantTime); + autoCommit(extraMetadata, result); + } else { + LOG.info("Auto commit disabled for " + instantTime); + } + } + + protected void autoCommit( + Option> extraMetadata, + HoodieWriteMetadata> result) { + this.txnManager.beginTransaction(Option.of( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime)), + lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); + try { + TransactionUtils.resolveWriteConflictIfAny( + table, this.txnManager.getCurrentTransactionOwner(), result.getCommitMetadata(), + config, this.txnManager.getLastCompletedTransactionOwner()); + commit(extraMetadata, result); + } finally { + this.txnManager.endTransaction(); + } + } + + /** + * Finalize Write operation. + * + * @param instantTime Instant Time + * @param stats Hoodie Write Stat + */ + protected void finalizeWrite( + String instantTime, List stats, + HoodieWriteMetadata> result) { + try { + Instant start = Instant.now(); + table.finalizeWrite(context, instantTime, stats); + result.setFinalizeDuration(Duration.between(start, Instant.now())); + } catch (HoodieIOException ioe) { + throw new HoodieCommitException( + "Failed to complete commit " + instantTime + " due to finalize errors.", ioe); + } + } + + protected Iterator> handleUpdateInternal( + HoodieMergeHandle upsertHandle, String fileId) throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); + } else { + getMergeHelper().runMerge(table, upsertHandle); + } + + // TODO(vc): This needs to be revisited + if (upsertHandle.getPartitionPath() == null) { + LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + + upsertHandle.writeStatuses()); + } + + return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java similarity index 63% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java index ac0f2596f490e..cf342620a5e77 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java @@ -18,9 +18,13 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -29,20 +33,23 @@ * * @param */ -public abstract class AbstractDeleteHelper { +public abstract class BaseDeleteHelper> { /** * Deduplicate Hoodie records, using the given deduplication function. * - * @param keys HoodieKeys to deduplicate - * @param table target Hoodie table for deduplicating + * @param keys HoodieKeys to deduplicate + * @param table target Hoodie table for deduplicating * @param parallelism parallelism or partitions to be used while reducing/deduplicating * @return HoodieKey already be deduplicated */ - public abstract K deduplicateKeys(K keys, HoodieTable table, int parallelism); + public abstract HoodieData deduplicateKeys( + HoodieData keys, + HoodieTable>, HoodieData, HoodieData> table, + int parallelism); - public abstract HoodieWriteMetadata execute(String instantTime, - K keys, HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - BaseCommitActionExecutor deleteExecutor); + public abstract HoodieWriteMetadata> execute( + String instantTime, HoodieData keys, HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + BaseCommitHelper commitHelper); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeletePartitionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeletePartitionHelper.java new file mode 100644 index 0000000000000..da7263306a4be --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeletePartitionHelper.java @@ -0,0 +1,37 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.io.Serializable; +import java.util.List; + +public abstract class BaseDeletePartitionHelper> implements Serializable { + + public abstract HoodieWriteMetadata> execute( + String instantTime, HoodieEngineContext context, HoodieTable table, + List partitions, BaseCommitHelper commitHelper); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java similarity index 97% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java index 59a3323bcb73e..7ddbe176daa95 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java @@ -47,11 +47,12 @@ /** * Helper to read records from previous version of base file and run Merge. */ -public abstract class AbstractMergeHelper { +public abstract class BaseMergeHelper { /** * Read records from previous version of base file and merge. - * @param table Hoodie Table + * + * @param table Hoodie Table * @param upsertHandle Merge Handle * @throws IOException in case of error */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java similarity index 64% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java index 455952ae5f7d7..89de5414b5ea9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java @@ -18,8 +18,11 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; @@ -29,30 +32,26 @@ import java.time.Duration; import java.time.Instant; -public abstract class AbstractWriteHelper { +public abstract class BaseWriteHelper> { - public HoodieWriteMetadata write(String instantTime, - I inputRecords, - HoodieEngineContext context, - HoodieTable table, - boolean shouldCombine, - int shuffleParallelism, - BaseCommitActionExecutor executor, - boolean performTagging) { + public HoodieWriteMetadata> write( + String instantTime, HoodieData> inputRecords, HoodieEngineContext context, + HoodieTable table, boolean shouldCombine, int shuffleParallelism, + BaseCommitHelper commitHelper, boolean performTagging) { try { // De-dupe/merge if needed - I dedupedRecords = + HoodieData> dedupedRecords = combineOnCondition(shouldCombine, inputRecords, shuffleParallelism, table); Instant lookupBegin = Instant.now(); - I taggedRecords = dedupedRecords; + HoodieData> taggedRecords = dedupedRecords; if (performTagging) { // perform index loop up to get existing location of records taggedRecords = tag(dedupedRecords, context, table); } Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); - HoodieWriteMetadata result = executor.execute(taggedRecords); + HoodieWriteMetadata> result = commitHelper.execute(taggedRecords); result.setIndexLookupDuration(indexLookupDuration); return result; } catch (Throwable e) { @@ -63,14 +62,11 @@ public HoodieWriteMetadata write(String instantTime, } } - private I tag( - I dedupedRecords, HoodieEngineContext context, HoodieTable table) { - // perform index loop up to get existing location of records - return table.getIndex().tagLocation(dedupedRecords, context, table); - } + protected abstract HoodieData> tag( + HoodieData> dedupedRecords, HoodieEngineContext context, HoodieTable table); - public I combineOnCondition( - boolean condition, I records, int parallelism, HoodieTable table) { + public HoodieData> combineOnCondition( + boolean condition, HoodieData> records, int parallelism, HoodieTable table) { return condition ? deduplicateRecords(records, table, parallelism) : records; } @@ -81,11 +77,11 @@ public I combineOnCondition( * @param parallelism parallelism or partitions to be used while reducing/deduplicating * @return Collection of HoodieRecord already be deduplicated */ - public I deduplicateRecords( - I records, HoodieTable table, int parallelism) { + public HoodieData> deduplicateRecords( + HoodieData> records, HoodieTable table, int parallelism) { return deduplicateRecords(records, table.getIndex(), parallelism); } - public abstract I deduplicateRecords( - I records, HoodieIndex index, int parallelism); + public abstract HoodieData> deduplicateRecords( + HoodieData> records, HoodieIndex index, int parallelism); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java new file mode 100644 index 0000000000000..d493861d85f74 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java @@ -0,0 +1,79 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.exception.HoodieInsertException; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.Map; + +public class BulkInsertCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + protected final BaseBulkInsertHelper bulkInsertHelper; + protected final HoodieData> inputRecords; + protected final Option>>> bulkInsertPartitioner; + + public BulkInsertCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, HoodieData> inputRecords, + Option>>> bulkInsertPartitioner, + BaseCommitHelper commitHelper, BaseBulkInsertHelper bulkInsertHelper) { + this(context, config, table, instantTime, inputRecords, bulkInsertPartitioner, + Option.empty(), commitHelper, bulkInsertHelper); + } + + public BulkInsertCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + String instantTime, HoodieData> inputRecords, + Option>>> bulkInsertPartitioner, + Option> extraMetadata, + BaseCommitHelper commitHelper, BaseBulkInsertHelper bulkInsertHelper) { + super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, + extraMetadata, commitHelper); + this.inputRecords = inputRecords; + this.bulkInsertPartitioner = bulkInsertPartitioner; + this.bulkInsertHelper = bulkInsertHelper; + } + + @Override + public HoodieWriteMetadata> execute() { + try { + return bulkInsertHelper.bulkInsert(inputRecords, instantTime, table, config, + true, bulkInsertPartitioner, commitHelper); + } catch (HoodieInsertException ie) { + throw ie; + } catch (Throwable e) { + throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..11b4980577adc --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java @@ -0,0 +1,82 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.exception.HoodieInsertException; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.Map; + +public class BulkInsertPreppedCommitActionExecutor> + extends BaseCommitActionExecutor { + + private final BaseCommitHelper commitHelper; + private final BaseBulkInsertHelper bulkInsertHelper; + private final HoodieData> preppedInputRecords; + private final Option>>> bulkInsertPartitioner; + + public BulkInsertPreppedCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable table, + String instantTime, HoodieData> preppedInputRecords, + Option>>> bulkInsertPartitioner, + BaseCommitHelper commitHelper, BaseBulkInsertHelper bulkInsertHelper) { + this(context, config, table, instantTime, preppedInputRecords, bulkInsertPartitioner, + Option.empty(), commitHelper, bulkInsertHelper); + } + + public BulkInsertPreppedCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + String instantTime, HoodieData> preppedInputRecords, + Option>>> bulkInsertPartitioner, + Option> extraMetadata, BaseCommitHelper commitHelper, + BaseBulkInsertHelper bulkInsertHelper) { + super(context, config, table, instantTime, WriteOperationType.BULK_INSERT_PREPPED, + extraMetadata, commitHelper); + this.preppedInputRecords = preppedInputRecords; + this.bulkInsertPartitioner = bulkInsertPartitioner; + this.commitHelper = commitHelper; + this.bulkInsertHelper = bulkInsertHelper; + } + + @Override + public HoodieWriteMetadata> execute() { + try { + return bulkInsertHelper.bulkInsert(preppedInputRecords, instantTime, table, config, + false, bulkInsertPartitioner, commitHelper); + } catch (HoodieInsertException ie) { + throw ie; + } catch (Throwable e) { + throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java new file mode 100644 index 0000000000000..c2a4620fea106 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java @@ -0,0 +1,51 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public class DeleteCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final BaseDeleteHelper deleteHelper; + private final HoodieData keys; + + public DeleteCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, HoodieData keys, + BaseCommitHelper commitHelper, BaseDeleteHelper deleteHelper) { + super(context, config, table, instantTime, WriteOperationType.DELETE, commitHelper); + this.keys = keys; + this.deleteHelper = deleteHelper; + } + + @Override + public HoodieWriteMetadata> execute() { + return deleteHelper.execute(instantTime, keys, context, config, table, commitHelper); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/DeletePartitionCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/DeletePartitionCommitActionExecutor.java new file mode 100644 index 0000000000000..b2bfde6926d9f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/DeletePartitionCommitActionExecutor.java @@ -0,0 +1,52 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; + +public class DeletePartitionCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final BaseDeletePartitionHelper deletePartitionHelper; + private List partitions; + + public DeletePartitionCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, List partitions, + BaseCommitHelper commitHelper, BaseDeletePartitionHelper deletePartitionHelper) { + super(context, config, table, instantTime, operationType, commitHelper); + this.partitions = partitions; + this.deletePartitionHelper = deletePartitionHelper; + } + + @Override + public HoodieWriteMetadata> execute() { + return deletePartitionHelper.execute(instantTime, context, table, partitions, commitHelper); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/ExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/ExecuteClusteringCommitActionExecutor.java new file mode 100644 index 0000000000000..06bd1c885eac6 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/ExecuteClusteringCommitActionExecutor.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public class ExecuteClusteringCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final BaseClusteringHelper clusteringHelper; + + public ExecuteClusteringCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable table, String instantTime, + BaseCommitHelper commitHelper, BaseClusteringHelper clusteringHelper) { + super(context, config, table, instantTime, WriteOperationType.CLUSTER, commitHelper); + this.clusteringHelper = clusteringHelper; + + } + + @Override + public HoodieWriteMetadata> execute() { + return clusteringHelper.execute( + context, instantTime, table, config, extraMetadata, commitHelper); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java new file mode 100644 index 0000000000000..b0f95089904a6 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java @@ -0,0 +1,53 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public class InsertCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final BaseWriteHelper writeHelper; + private HoodieData> inputRecords; + + public InsertCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + HoodieData> inputRecords, + BaseCommitHelper commitHelper, BaseWriteHelper writeHelper) { + super(context, config, table, instantTime, operationType, commitHelper); + this.inputRecords = inputRecords; + this.writeHelper = writeHelper; + } + + @Override + public HoodieWriteMetadata> execute() { + return writeHelper.write(instantTime, inputRecords, context, table, + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), commitHelper, false); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..b08ba9df3be06 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java @@ -0,0 +1,49 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public class InsertPreppedCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final HoodieData> preppedRecords; + + public InsertPreppedCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable table, String instantTime, + HoodieData> preppedRecords, BaseCommitHelper commitHelper) { + super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED, commitHelper); + this.preppedRecords = preppedRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return commitHelper.execute(preppedRecords); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java new file mode 100644 index 0000000000000..18843f5524fc1 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java @@ -0,0 +1,52 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public class UpsertCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final BaseWriteHelper writeHelper; + private HoodieData> inputRecords; + + public UpsertCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable table, + String instantTime, HoodieData> inputRecords, + BaseCommitHelper commitHelper, BaseWriteHelper writeHelper) { + super(context, config, table, instantTime, WriteOperationType.UPSERT, commitHelper); + this.inputRecords = inputRecords; + this.writeHelper = writeHelper; + } + + @Override + public HoodieWriteMetadata> execute() { + return writeHelper.write(instantTime, inputRecords, context, table, + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), commitHelper, true); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..2a55e30382d1e --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java @@ -0,0 +1,49 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +public class UpsertPreppedCommitActionExecutor> + extends BaseCommitActionExecutor>> { + + private final HoodieData> preppedRecords; + + public UpsertPreppedCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable table, String instantTime, + HoodieData> preppedRecords, BaseCommitHelper commitHelper) { + super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED, commitHelper); + this.preppedRecords = preppedRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return commitHelper.execute(preppedRecords); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java index 174122c68180b..76941b62554ab 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java @@ -39,6 +39,8 @@ import java.util.stream.Stream; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.util.FlinkClientUtil; import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper; @@ -68,10 +70,19 @@ public HoodieFlinkEngineContext(SerializableConfiguration hadoopConf, TaskContex this.runtimeContext = ((FlinkTaskContextSupplier) taskContextSupplier).getFlinkRuntimeContext(); } + public static List getList(HoodieData hoodieData) { + return ((HoodieListData) hoodieData).get(); + } + public RuntimeContext getRuntimeContext() { return this.runtimeContext; } + @Override + public List map(HoodieData data, SerializableFunction func) { + return getList(data).stream().map(throwingMapWrapper(func)).collect(Collectors.toList()); + } + @Override public List map(List data, SerializableFunction func, int parallelism) { return data.stream().parallel().map(throwingMapWrapper(func)).collect(Collectors.toList()); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 93785b919bcb1..f1f2ccf3212b0 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -32,10 +32,12 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.HoodieCreateHandle; @@ -46,14 +48,16 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.FlinkCleanActionExecutor; import org.apache.hudi.table.action.clean.FlinkScheduleCleanActionExecutor; -import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor; -import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.FlinkInsertOverwriteCommitActionExecutor; -import org.apache.hudi.table.action.commit.FlinkInsertOverwriteTableCommitActionExecutor; -import org.apache.hudi.table.action.commit.FlinkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.FlinkCommitHelper; +import org.apache.hudi.table.action.commit.FlinkDeleteHelper; +import org.apache.hudi.table.action.commit.FlinkInsertOverwriteCommitHelper; import org.apache.hudi.table.action.commit.FlinkMergeHelper; -import org.apache.hudi.table.action.commit.FlinkUpsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.FlinkUpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.FlinkWriteHelper; +import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; @@ -74,7 +78,7 @@ *

* UPDATES - Produce a new version of the file, just replacing the updated records with new values */ -public class HoodieFlinkCopyOnWriteTable extends HoodieFlinkTable { +public class HoodieFlinkCopyOnWriteTable> extends HoodieFlinkTable { private static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkCopyOnWriteTable.class); @@ -99,7 +103,11 @@ public HoodieWriteMetadata> upsert( HoodieWriteHandle writeHandle, String instantTime, List> records) { - return new FlinkUpsertCommitActionExecutor<>(context, writeHandle, config, this, instantTime, records).execute(); + return convertMetadata(new UpsertCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(records), + new FlinkCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.UPSERT), + FlinkWriteHelper.newInstance()).execute()); } /** @@ -119,7 +127,11 @@ public HoodieWriteMetadata> insert( HoodieWriteHandle writeHandle, String instantTime, List> records) { - return new FlinkInsertCommitActionExecutor<>(context, writeHandle, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT, HoodieListData.of(records), + new FlinkCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.INSERT), + FlinkWriteHelper.newInstance()).execute()); } /** @@ -140,7 +152,11 @@ public HoodieWriteMetadata> delete( HoodieWriteHandle writeHandle, String instantTime, List keys) { - return new FlinkDeleteCommitActionExecutor<>(context, writeHandle, config, this, instantTime, keys).execute(); + return convertMetadata(new DeleteCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(keys), + new FlinkCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.DELETE), + FlinkDeleteHelper.newInstance()).execute()); } /** @@ -161,7 +177,10 @@ public HoodieWriteMetadata> upsertPrepped( HoodieWriteHandle writeHandle, String instantTime, List> preppedRecords) { - return new FlinkUpsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new UpsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + new FlinkCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.UPSERT_PREPPED)).execute()); } /** @@ -182,7 +201,10 @@ public HoodieWriteMetadata> insertPrepped( HoodieWriteHandle writeHandle, String instantTime, List> preppedRecords) { - return new FlinkInsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new InsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + new FlinkCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.INSERT_PREPPED)).execute()); } @Override @@ -191,7 +213,12 @@ public HoodieWriteMetadata> insertOverwrite( HoodieWriteHandle writeHandle, String instantTime, List> records) { - return new FlinkInsertOverwriteCommitActionExecutor(context, writeHandle, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE, + HoodieListData.of(records), + new FlinkInsertOverwriteCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE), + FlinkWriteHelper.newInstance()).execute()); } @Override @@ -200,7 +227,12 @@ public HoodieWriteMetadata> insertOverwriteTable( HoodieWriteHandle writeHandle, String instantTime, List> records) { - return new FlinkInsertOverwriteTableCommitActionExecutor(context, writeHandle, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, + HoodieListData.of(records), + new FlinkInsertOverwriteCommitHelper<>( + context, writeHandle, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE), + FlinkWriteHelper.newInstance()).execute()); } @Override @@ -290,8 +322,8 @@ public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { } /** - * @param context HoodieEngineContext - * @param instantTime Instant Time for scheduling cleaning + * @param context HoodieEngineContext + * @param instantTime Instant Time for scheduling cleaning * @param extraMetadata additional metadata to write into plan * @return */ diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java index f4a4b0eb48982..4a23037eb5dbd 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -25,17 +25,21 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.io.FlinkAppendHandle; import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.delta.FlinkUpsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.commit.delta.FlinkUpsertPreppedDeltaCommitActionExecutor; +import org.apache.hudi.table.action.commit.FlinkWriteHelper; +import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.delta.FlinkDeltaCommitHelper; import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor; import org.apache.hudi.table.action.compact.FlinkScheduleCompactionActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; @@ -44,7 +48,7 @@ import java.util.List; import java.util.Map; -public class HoodieFlinkMergeOnReadTable +public class HoodieFlinkMergeOnReadTable> extends HoodieFlinkCopyOnWriteTable { HoodieFlinkMergeOnReadTable( @@ -63,7 +67,11 @@ public HoodieWriteMetadata> upsert( ValidationUtils.checkArgument(writeHandle instanceof FlinkAppendHandle, "MOR write handle should always be a FlinkAppendHandle"); FlinkAppendHandle appendHandle = (FlinkAppendHandle) writeHandle; - return new FlinkUpsertDeltaCommitActionExecutor<>(context, appendHandle, config, this, instantTime, hoodieRecords).execute(); + return convertMetadata(new UpsertCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(hoodieRecords), + new FlinkDeltaCommitHelper<>( + context, appendHandle, config, this, instantTime, WriteOperationType.UPSERT), + FlinkWriteHelper.newInstance()).execute()); } @Override @@ -75,7 +83,10 @@ public HoodieWriteMetadata> upsertPrepped( ValidationUtils.checkArgument(writeHandle instanceof FlinkAppendHandle, "MOR write handle should always be a FlinkAppendHandle"); FlinkAppendHandle appendHandle = (FlinkAppendHandle) writeHandle; - return new FlinkUpsertPreppedDeltaCommitActionExecutor<>(context, appendHandle, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new UpsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + new FlinkDeltaCommitHelper<>( + context, appendHandle, config, this, instantTime, WriteOperationType.UPSERT_PREPPED)).execute()); } @Override @@ -86,7 +97,11 @@ public HoodieWriteMetadata> insert( List> hoodieRecords) { if (writeHandle instanceof FlinkAppendHandle) { FlinkAppendHandle appendHandle = (FlinkAppendHandle) writeHandle; - return new FlinkUpsertDeltaCommitActionExecutor<>(context, appendHandle, config, this, instantTime, hoodieRecords).execute(); + return convertMetadata(new UpsertCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(hoodieRecords), + new FlinkDeltaCommitHelper<>( + context, appendHandle, config, this, instantTime, WriteOperationType.UPSERT), + FlinkWriteHelper.newInstance()).execute()); } else { return super.insert(context, writeHandle, instantTime, hoodieRecords); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 3e26025c258bb..eeebe34f41247 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -28,9 +28,13 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.FlinkHoodieIndex; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.FlinkCommitHelper; import java.util.List; @@ -62,8 +66,40 @@ public static HoodieFlinkTable create(HoodieW } } + public static Option>>> convertBulkInsertPartitioner( + Option>>> userDefinedBulkInsertPartitioner) { + Option>>> partitionerOption = Option.empty(); + if (userDefinedBulkInsertPartitioner.isPresent()) { + partitionerOption = Option.of(convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner.get())); + } + return partitionerOption; + } + + public static BulkInsertPartitioner>> convertBulkInsertPartitioner( + BulkInsertPartitioner>> bulkInsertPartitioner) { + return new BulkInsertPartitioner>>() { + @Override + public HoodieData> repartitionRecords( + HoodieData> records, int outputSparkPartitions) { + return HoodieListData.of(bulkInsertPartitioner.repartitionRecords( + FlinkCommitHelper.getList(records), outputSparkPartitions)); + } + + @Override + public boolean arePartitionRecordsSorted() { + return bulkInsertPartitioner.arePartitionRecordsSorted(); + } + }; + } + + public static HoodieWriteMetadata> convertMetadata( + HoodieWriteMetadata> metadata) { + return metadata.clone(FlinkCommitHelper.getList(metadata.getWriteStatuses())); + } + @Override - protected HoodieIndex>, List, List> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { + protected HoodieIndex>, List, List> getIndex( + HoodieWriteConfig config, HoodieEngineContext context) { return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java deleted file mode 100644 index 5cfd28be2c1dc..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ /dev/null @@ -1,245 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.CommitUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCommitException; -import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.execution.FlinkLazyInsertIterable; -import org.apache.hudi.io.ExplicitWriteHandleFactory; -import org.apache.hudi.io.HoodieCreateHandle; -import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.util.Collections; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -/** - * With {@code org.apache.hudi.operator.partitioner.BucketAssigner}, each hoodie record - * is tagged with a bucket ID (partition path + fileID) in streaming way. All the records consumed by this - * executor should be tagged with bucket IDs and belong to one data bucket. - * - *

These bucket IDs make it possible to shuffle the records first by the bucket ID - * (see org.apache.hudi.operator.partitioner.BucketAssignerFunction), and this executor - * only needs to handle the data buffer that belongs to one data bucket once at a time. So there is no need to - * partition the buffer. - * - *

Computing the records batch locations all at a time is a pressure to the engine, - * we should avoid that in streaming system. - */ -public abstract class BaseFlinkCommitActionExecutor extends - BaseCommitActionExecutor>, List, List, HoodieWriteMetadata> { - - private static final Logger LOG = LogManager.getLogger(BaseFlinkCommitActionExecutor.class); - - protected HoodieWriteHandle writeHandle; - - public BaseFlinkCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType) { - this(context, writeHandle, config, table, instantTime, operationType, Option.empty()); - } - - public BaseFlinkCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType, - Option extraMetadata) { - super(context, config, table, instantTime, operationType, extraMetadata); - this.writeHandle = writeHandle; - } - - @Override - public HoodieWriteMetadata> execute(List> inputRecords) { - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); - - List writeStatuses = new LinkedList<>(); - final HoodieRecord record = inputRecords.get(0); - final String partitionPath = record.getPartitionPath(); - final String fileId = record.getCurrentLocation().getFileId(); - final BucketType bucketType = record.getCurrentLocation().getInstantTime().equals("I") - ? BucketType.INSERT - : BucketType.UPDATE; - handleUpsertPartition( - instantTime, - partitionPath, - fileId, - bucketType, - inputRecords.iterator()) - .forEachRemaining(writeStatuses::addAll); - setUpWriteMetadata(writeStatuses, result); - return result; - } - - protected void setUpWriteMetadata( - List statuses, - HoodieWriteMetadata> result) { - // No need to update the index because the update happens before the write. - result.setWriteStatuses(statuses); - result.setIndexUpdateDuration(Duration.ZERO); - } - - @Override - protected String getCommitActionType() { - return table.getMetaClient().getCommitActionType(); - } - - @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { - commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList())); - } - - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { - String actionType = getCommitActionType(); - LOG.info("Committing " + instantTime + ", action Type " + actionType); - result.setCommitted(true); - result.setWriteStats(writeStats); - // Finalize write - finalizeWrite(instantTime, writeStats, result); - syncTableMetadata(); - try { - LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), - extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); - - activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - LOG.info("Committed " + instantTime); - result.setCommitMetadata(Option.of(metadata)); - } catch (IOException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); - } - } - - @Override - protected boolean isWorkloadProfileNeeded() { - return true; - } - - @SuppressWarnings("unchecked") - protected Iterator> handleUpsertPartition( - String instantTime, - String partitionPath, - String fileIdHint, - BucketType bucketType, - Iterator recordItr) { - try { - if (this.writeHandle instanceof HoodieCreateHandle) { - // During one checkpoint interval, an insert record could also be updated, - // for example, for an operation sequence of a record: - // I, U, | U, U - // - batch1 - | - batch2 - - // the first batch(batch1) operation triggers an INSERT bucket, - // the second batch batch2 tries to reuse the same bucket - // and append instead of UPDATE. - return handleInsert(fileIdHint, recordItr); - } else if (this.writeHandle instanceof HoodieMergeHandle) { - return handleUpdate(partitionPath, fileIdHint, recordItr); - } else { - switch (bucketType) { - case INSERT: - return handleInsert(fileIdHint, recordItr); - case UPDATE: - return handleUpdate(partitionPath, fileIdHint, recordItr); - default: - throw new AssertionError(); - } - } - } catch (Throwable t) { - String msg = "Error upsetting bucketType " + bucketType + " for partition :" + partitionPath; - LOG.error(msg, t); - throw new HoodieUpsertException(msg, t); - } - } - - @Override - public Iterator> handleUpdate(String partitionPath, String fileId, - Iterator> recordItr) - throws IOException { - // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records - if (!recordItr.hasNext()) { - LOG.info("Empty partition with fileId => " + fileId); - return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); - } - // these are updates - HoodieMergeHandle upsertHandle = (HoodieMergeHandle) this.writeHandle; - return handleUpdateInternal(upsertHandle, fileId); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) - throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException( - "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); - } else { - FlinkMergeHelper.newInstance().runMerge(table, upsertHandle); - } - - // TODO(vc): This needs to be revisited - if (upsertHandle.getPartitionPath() == null) { - LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.writeStatuses()); - } - - return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); - } - - @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { - // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records - if (!recordItr.hasNext()) { - LOG.info("Empty partition"); - return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); - } - return new FlinkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, - taskContextSupplier, new ExplicitWriteHandleFactory<>(writeHandle)); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkCommitHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkCommitHelper.java new file mode 100644 index 0000000000000..6872d1db27fe5 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkCommitHelper.java @@ -0,0 +1,170 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.execution.FlinkLazyInsertIterable; +import org.apache.hudi.io.ExplicitWriteHandleFactory; +import org.apache.hudi.io.HoodieCreateHandle; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.HoodieWriteHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.time.Duration; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +public class FlinkCommitHelper> extends BaseCommitHelper { + + private static final Logger LOG = LogManager.getLogger(FlinkCommitHelper.class); + protected HoodieWriteHandle writeHandle; + + public FlinkCommitHelper( + HoodieEngineContext context, HoodieWriteHandle writeHandle, + HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + this(context, writeHandle, config, table, instantTime, operationType, Option.empty()); + } + + public FlinkCommitHelper( + HoodieEngineContext context, HoodieWriteHandle writeHandle, + HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + super(context, config, table, instantTime, operationType, extraMetadata); + this.writeHandle = writeHandle; + } + + public static List getList(HoodieData hoodieData) { + return ((HoodieListData) hoodieData).get(); + } + + @Override + public void init(HoodieWriteConfig config) { + // No OP + } + + @Override + public HoodieWriteMetadata> execute(HoodieData> inputRecords) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + + List writeStatuses = new LinkedList<>(); + final HoodieRecord record = getList(inputRecords).get(0); + final String partitionPath = record.getPartitionPath(); + final String fileId = record.getCurrentLocation().getFileId(); + final BucketType bucketType = record.getCurrentLocation().getInstantTime().equals("I") + ? BucketType.INSERT + : BucketType.UPDATE; + handleUpsertPartition( + instantTime, + partitionPath, + fileId, + bucketType, + getList(inputRecords).iterator()) + .forEachRemaining(writeStatuses::addAll); + setUpWriteMetadata(writeStatuses, result); + return result; + } + + protected void setUpWriteMetadata( + List statuses, + HoodieWriteMetadata> result) { + // No need to update the index because the update happens before the write. + result.setWriteStatuses(HoodieListData.of(statuses)); + result.setIndexUpdateDuration(Duration.ZERO); + } + + protected Iterator> handleUpsertPartition( + String instantTime, + String partitionPath, + String fileIdHint, + BucketType bucketType, + Iterator recordItr) { + try { + if (this.writeHandle instanceof HoodieCreateHandle) { + // During one checkpoint interval, an insert record could also be updated, + // for example, for an operation sequence of a record: + // I, U, | U, U + // - batch1 - | - batch2 - + // the first batch(batch1) operation triggers an INSERT bucket, + // the second batch batch2 tries to reuse the same bucket + // and append instead of UPDATE. + return handleInsert(fileIdHint, recordItr); + } else if (this.writeHandle instanceof HoodieMergeHandle) { + return handleUpdate(partitionPath, fileIdHint, recordItr); + } else { + switch (bucketType) { + case INSERT: + return handleInsert(fileIdHint, recordItr); + case UPDATE: + return handleUpdate(partitionPath, fileIdHint, recordItr); + default: + throw new AssertionError(); + } + } + } catch (Throwable t) { + String msg = "Error upsetting bucketType " + bucketType + " for partition :" + partitionPath; + LOG.error(msg, t); + throw new HoodieUpsertException(msg, t); + } + } + + @Override + public Iterator> handleInsert(String idPfx, Iterator> recordItr) throws Exception { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + LOG.info("Empty partition"); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } + return new FlinkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, + context.getTaskContextSupplier(), new ExplicitWriteHandleFactory<>(writeHandle)); + } + + @Override + protected void syncTableMetadata() { + // No OP + } + + @Override + protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { + return (HoodieMergeHandle) this.writeHandle; + } + + @Override + protected BaseMergeHelper getMergeHelper() { + return FlinkMergeHelper.newInstance(); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java deleted file mode 100644 index 2064be3e67113..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java +++ /dev/null @@ -1,50 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkDeleteCommitActionExecutor> extends BaseFlinkCommitActionExecutor { - private final List keys; - - public FlinkDeleteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List keys) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.DELETE); - this.keys = keys; - } - - @Override - public HoodieWriteMetadata> execute() { - return FlinkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java index 185482642ead3..083a8bffab5c1 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java @@ -23,8 +23,11 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -41,8 +44,7 @@ import java.util.stream.Collectors; @SuppressWarnings("checkstyle:LineLength") -public class FlinkDeleteHelper extends - AbstractDeleteHelper>, List, List, R> { +public class FlinkDeleteHelper> extends BaseDeleteHelper { private FlinkDeleteHelper() { } @@ -56,60 +58,66 @@ public static FlinkDeleteHelper newInstance() { } @Override - public List deduplicateKeys(List keys, HoodieTable>, List, List> table, int parallelism) { + public HoodieData deduplicateKeys( + HoodieData keys, + HoodieTable>, HoodieData, HoodieData> table, + int parallelism) { + List keysList = ((HoodieListData) keys).get(); boolean isIndexingGlobal = table.getIndex().isGlobal(); if (isIndexingGlobal) { - HashSet recordKeys = keys.stream().map(HoodieKey::getRecordKey).collect(Collectors.toCollection(HashSet::new)); + HashSet recordKeys = keysList.stream().map(HoodieKey::getRecordKey) + .collect(Collectors.toCollection(HashSet::new)); List deduplicatedKeys = new LinkedList<>(); - keys.forEach(x -> { + keysList.forEach(x -> { if (recordKeys.contains(x.getRecordKey())) { deduplicatedKeys.add(x); } }); - return deduplicatedKeys; + return HoodieListData.of(deduplicatedKeys); } else { - HashSet set = new HashSet<>(keys); - keys.clear(); - keys.addAll(set); + HashSet set = new HashSet<>(keysList); + keysList.clear(); + keysList.addAll(set); return keys; } } @Override - public HoodieWriteMetadata> execute(String instantTime, - List keys, - HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable>, List, List> table, - BaseCommitActionExecutor>, List, List, R> deleteExecutor) { + public HoodieWriteMetadata> execute( + String instantTime, HoodieData keys, HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + BaseCommitHelper commitHelper) { try { - HoodieWriteMetadata> result = null; - List dedupedKeys = keys; + HoodieWriteMetadata> result = null; + HoodieData dedupedKeys = keys; final int parallelism = config.getDeleteShuffleParallelism(); if (config.shouldCombineBeforeDelete()) { // De-dupe/merge if needed dedupedKeys = deduplicateKeys(keys, table, parallelism); } - List> dedupedRecords = - dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); + List dedupedRecords = FlinkCommitHelper.getList(dedupedKeys).stream().map( + key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); Instant beginTag = Instant.now(); // perform index look up to get existing location of records - List> taggedRecords = - table.getIndex().tagLocation(dedupedRecords, context, table); + HoodieData> taggedRecords = HoodieListData.of((List>) + table.getIndex().tagLocation(dedupedRecords, context, table)); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records - List> taggedValidRecords = taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).collect(Collectors.toList()); + List> taggedValidRecords = + FlinkCommitHelper.getList(taggedRecords).stream() + .filter(HoodieRecord::isCurrentLocationKnown).collect(Collectors.toList()); if (!taggedValidRecords.isEmpty()) { - result = deleteExecutor.execute(taggedValidRecords); + result = commitHelper.execute(HoodieListData.of(taggedValidRecords)); result.setIndexLookupDuration(tagLocationDuration); } else { // if entire set of keys are non existent - deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + commitHelper.saveWorkloadProfileMetadataToInflight( + new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); result = new HoodieWriteMetadata<>(); - result.setWriteStatuses(Collections.EMPTY_LIST); - deleteExecutor.commitOnAutoCommit(result); + result.setWriteStatuses(HoodieListData.of(Collections.EMPTY_LIST)); + commitHelper.commitOnAutoCommit(result); } return result; } catch (Throwable e) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java deleted file mode 100644 index 041598314f343..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java +++ /dev/null @@ -1,52 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkInsertCommitActionExecutor> extends BaseFlinkCommitActionExecutor { - - private List> inputRecords; - - public FlinkInsertCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.INSERT); - this.inputRecords = inputRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java deleted file mode 100644 index 583e0b6a94b98..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java +++ /dev/null @@ -1,69 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkInsertOverwriteCommitActionExecutor> - extends BaseFlinkCommitActionExecutor { - - protected List> inputRecords; - - public FlinkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - this(context, writeHandle, config, table, instantTime, inputRecords, WriteOperationType.INSERT_OVERWRITE); - } - - public FlinkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords, - WriteOperationType writeOperationType) { - super(context, writeHandle, config, table, instantTime, writeOperationType); - this.inputRecords = inputRecords; - } - - @Override - protected String getCommitActionType() { - return HoodieTimeline.REPLACE_COMMIT_ACTION; - } - - @Override - public HoodieWriteMetadata> execute() { - return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitHelper.java new file mode 100644 index 0000000000000..54bc97d632c5c --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitHelper.java @@ -0,0 +1,53 @@ +/* + * 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.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.HoodieWriteHandle; +import org.apache.hudi.table.HoodieTable; + +import java.util.Map; + +public class FlinkInsertOverwriteCommitHelper> extends FlinkCommitHelper { + public FlinkInsertOverwriteCommitHelper( + HoodieEngineContext context, HoodieWriteHandle writeHandle, + HoodieWriteConfig config, + HoodieTable table, String instantTime, WriteOperationType operationType) { + this(context, writeHandle, config, table, instantTime, operationType, Option.empty()); + } + + public FlinkInsertOverwriteCommitHelper( + HoodieEngineContext context, HoodieWriteHandle writeHandle, + HoodieWriteConfig config, + HoodieTable table, String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + super(context, writeHandle, config, table, instantTime, operationType, extraMetadata); + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java deleted file mode 100644 index a31679b63f709..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java +++ /dev/null @@ -1,50 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkInsertOverwriteTableCommitActionExecutor> - extends FlinkInsertOverwriteCommitActionExecutor { - - public FlinkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - super(context, writeHandle, config, table, instantTime, inputRecords, WriteOperationType.INSERT_OVERWRITE_TABLE); - } - - @Override - public HoodieWriteMetadata> execute() { - return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java deleted file mode 100644 index 459a6dbc8f672..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkInsertPreppedCommitActionExecutor> extends BaseFlinkCommitActionExecutor { - - private final List> preppedRecords; - - public FlinkInsertPreppedCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> preppedRecords) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.INSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java index 5ed6d5d529ba3..c88b4ee66a098 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java @@ -45,7 +45,7 @@ import scala.collection.immutable.List; -public class FlinkMergeHelper extends AbstractMergeHelper>, +public class FlinkMergeHelper extends BaseMergeHelper>, List, List> { private FlinkMergeHelper() { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java deleted file mode 100644 index 5859bb585fcd9..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java +++ /dev/null @@ -1,52 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkUpsertCommitActionExecutor> extends BaseFlinkCommitActionExecutor { - - private List> inputRecords; - - public FlinkUpsertCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT); - this.inputRecords = inputRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java deleted file mode 100644 index 42d932a2bd7f9..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkUpsertPreppedCommitActionExecutor> extends BaseFlinkCommitActionExecutor { - - private final List> preppedRecords; - - public FlinkUpsertPreppedCommitActionExecutor(HoodieEngineContext context, - HoodieWriteHandle writeHandle, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> preppedRecords) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java index 5cb1b80a5136e..37f1c67e9de3f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; @@ -46,8 +48,7 @@ *

Computing the records batch locations all at a time is a pressure to the engine, * we should avoid that in streaming system. */ -public class FlinkWriteHelper extends AbstractWriteHelper>, - List, List, R> { +public class FlinkWriteHelper> extends BaseWriteHelper { private FlinkWriteHelper() { } @@ -61,14 +62,15 @@ public static FlinkWriteHelper newInstance() { } @Override - public HoodieWriteMetadata> write(String instantTime, List> inputRecords, HoodieEngineContext context, - HoodieTable>, List, List> table, boolean shouldCombine, int shuffleParallelism, - BaseCommitActionExecutor>, List, List, R> executor, boolean performTagging) { + public HoodieWriteMetadata> write( + String instantTime, HoodieData> inputRecords, HoodieEngineContext context, + HoodieTable table, boolean shouldCombine, int shuffleParallelism, + BaseCommitHelper commitHelper, boolean performTagging) { try { Instant lookupBegin = Instant.now(); Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); - HoodieWriteMetadata> result = executor.execute(inputRecords); + HoodieWriteMetadata> result = commitHelper.execute(inputRecords); result.setIndexLookupDuration(indexLookupDuration); return result; } catch (Throwable e) { @@ -80,30 +82,43 @@ public HoodieWriteMetadata> write(String instantTime, List> deduplicateRecords(List> records, - HoodieIndex>, List, List> index, - int parallelism) { - Map>>> keyedRecords = records.stream().map(record -> { - // If index used is global, then records are expected to differ in their partitionPath - final Object key = record.getKey().getRecordKey(); - return Pair.of(key, record); - }).collect(Collectors.groupingBy(Pair::getLeft)); + protected HoodieData> tag( + HoodieData> dedupedRecords, HoodieEngineContext context, HoodieTable table) { + // perform index loop up to get existing location of records + return HoodieListData.of((List>) + table.getIndex().tagLocation(FlinkCommitHelper.getList(dedupedRecords), context, table)); + } + + @Override + public HoodieData> deduplicateRecords( + HoodieData> records, HoodieIndex index, int parallelism) { + Map>>> keyedRecords = + FlinkCommitHelper.getList(records).stream().map(record -> { + // If index used is global, then records are expected to differ in their partitionPath + final Object key = record.getKey().getRecordKey(); + return Pair.of(key, record); + }).collect(Collectors.groupingBy(Pair::getLeft)); - return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { - final T data1 = rec1.getData(); - final T data2 = rec2.getData(); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything + // so pick it from one of the records. + // reuse the location from the first record. + return HoodieListData.of(keyedRecords.values().stream() + .map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { + final T data1 = rec1.getData(); + final T data2 = rec2.getData(); - @SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1); - // we cannot allow the user to change the key or partitionPath, since that will affect - // everything - // so pick it from one of the records. - boolean choosePrev = data1.equals(reducedData); - HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey(); - HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation(); - HoodieRecord hoodieRecord = new HoodieRecord<>(reducedKey, reducedData, operation); - // reuse the location from the first record. - hoodieRecord.setCurrentLocation(rec1.getCurrentLocation()); - return hoodieRecord; - }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); + @SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything + // so pick it from one of the records. + boolean choosePrev = data1.equals(reducedData); + HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey(); + HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation(); + HoodieRecord hoodieRecord = new HoodieRecord<>(reducedKey, reducedData, operation); + // reuse the location from the first record. + hoodieRecord.setCurrentLocation(rec1.getCurrentLocation()); + return hoodieRecord; + }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList())); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkDeltaCommitHelper.java similarity index 59% rename from hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java rename to hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkDeltaCommitHelper.java index 1fe98204b1cb1..8a7761156286a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkDeltaCommitHelper.java @@ -7,13 +7,14 @@ * "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 + * 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. + * 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.delta; @@ -27,27 +28,24 @@ import org.apache.hudi.execution.FlinkLazyInsertIterable; import org.apache.hudi.io.ExplicitWriteHandleFactory; import org.apache.hudi.io.FlinkAppendHandle; +import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.commit.BaseFlinkCommitActionExecutor; +import org.apache.hudi.table.action.commit.FlinkCommitHelper; import java.util.Collections; import java.util.Iterator; import java.util.List; -public abstract class BaseFlinkDeltaCommitActionExecutor> - extends BaseFlinkCommitActionExecutor { - - public BaseFlinkDeltaCommitActionExecutor(HoodieEngineContext context, - FlinkAppendHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType) { +public class FlinkDeltaCommitHelper> extends FlinkCommitHelper { + public FlinkDeltaCommitHelper( + HoodieEngineContext context, HoodieWriteHandle writeHandle, HoodieWriteConfig config, + HoodieTable table, String instantTime, WriteOperationType operationType) { super(context, writeHandle, config, table, instantTime, operationType); } @Override - public Iterator> handleUpdate(String partitionPath, String fileId, Iterator> recordItr) { + public Iterator> handleUpdate( + String partitionPath, String fileId, Iterator> recordItr) { FlinkAppendHandle appendHandle = (FlinkAppendHandle) writeHandle; appendHandle.doAppend(); List writeStatuses = appendHandle.close(); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java deleted file mode 100644 index 5fdf46f6ebcbe..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java +++ /dev/null @@ -1,52 +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.delta; - -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.FlinkAppendHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.FlinkWriteHelper; - -import java.util.List; - -public class FlinkUpsertDeltaCommitActionExecutor> - extends BaseFlinkDeltaCommitActionExecutor { - private final List> inputRecords; - - public FlinkUpsertDeltaCommitActionExecutor(HoodieEngineContext context, - FlinkAppendHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT); - this.inputRecords = inputRecords; - } - - @Override - public HoodieWriteMetadata execute() { - return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); - } -} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java deleted file mode 100644 index 493c894c8a96c..0000000000000 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java +++ /dev/null @@ -1,52 +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.delta; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.FlinkAppendHandle; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class FlinkUpsertPreppedDeltaCommitActionExecutor> - extends BaseFlinkDeltaCommitActionExecutor { - - private final List> preppedRecords; - - public FlinkUpsertPreppedDeltaCommitActionExecutor(HoodieEngineContext context, - FlinkAppendHandle writeHandle, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> preppedRecords) { - super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java index 4cdbff264bdbc..00f1678d9482d 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java @@ -30,6 +30,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import java.util.List; import java.util.Map; @@ -57,6 +59,15 @@ public HoodieJavaEngineContext(Configuration conf, TaskContextSupplier taskConte super(new SerializableConfiguration(conf), taskContextSupplier); } + public static List getList(HoodieData hoodieData) { + return ((HoodieListData) hoodieData).get(); + } + + @Override + public List map(HoodieData data, SerializableFunction func) { + return getList(data).stream().map(throwingMapWrapper(func)).collect(Collectors.toList()); + } + @Override public List map(List data, SerializableFunction func, int parallelism) { return data.stream().parallel().map(throwingMapWrapper(func)).collect(toList()); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index 72d63d5a005c0..d4d4e5c2d2c3c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -32,24 +32,30 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.JavaCleanActionExecutor; import org.apache.hudi.table.action.clean.JavaScheduleCleanActionExecutor; -import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaBulkInsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaInsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaInsertOverwriteCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaInsertOverwriteTableCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaInsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaUpsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.JavaUpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaBulkInsertHelper; +import org.apache.hudi.table.action.commit.JavaCommitHelper; +import org.apache.hudi.table.action.commit.JavaDeleteHelper; +import org.apache.hudi.table.action.commit.JavaInsertOverwriteCommitHelper; +import org.apache.hudi.table.action.commit.JavaInsertOverwriteTableCommitHelper; +import org.apache.hudi.table.action.commit.JavaWriteHelper; +import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.restore.JavaCopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; @@ -58,7 +64,7 @@ import java.util.List; import java.util.Map; -public class HoodieJavaCopyOnWriteTable extends HoodieJavaTable { +public class HoodieJavaCopyOnWriteTable> extends HoodieJavaTable { protected HoodieJavaCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { @@ -69,16 +75,20 @@ protected HoodieJavaCopyOnWriteTable(HoodieWriteConfig config, public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, List> records) { - return new JavaUpsertCommitActionExecutor<>(context, config, - this, instantTime, records).execute(); + return convertMetadata(new UpsertCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(records), + new JavaCommitHelper<>(context, config, this, instantTime, WriteOperationType.UPSERT), + JavaWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, List> records) { - return new JavaInsertCommitActionExecutor<>(context, config, - this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT, HoodieListData.of(records), + new JavaCommitHelper<>(context, config, this, instantTime, WriteOperationType.INSERT), + JavaWriteHelper.newInstance()).execute()); } @Override @@ -86,15 +96,21 @@ public HoodieWriteMetadata> bulkInsert(HoodieEngineContext con String instantTime, List> records, Option>>> bulkInsertPartitioner) { - return new JavaBulkInsertCommitActionExecutor((HoodieJavaEngineContext) context, config, - this, instantTime, records, bulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(records), + convertBulkInsertPartitioner(bulkInsertPartitioner), + new JavaCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT), + JavaBulkInsertHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, List keys) { - return new JavaDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute(); + return convertMetadata(new DeleteCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(keys), + new JavaCommitHelper<>(context, config, this, instantTime, WriteOperationType.DELETE), + JavaDeleteHelper.newInstance()).execute()); } @Override @@ -106,17 +122,20 @@ public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { - return new JavaUpsertPreppedCommitActionExecutor<>((HoodieJavaEngineContext) context, config, - this, instantTime, preppedRecords).execute(); - + return convertMetadata(new UpsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + new JavaCommitHelper<>( + context, config, this, instantTime, WriteOperationType.UPSERT_PREPPED)).execute()); } @Override public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { - return new JavaInsertPreppedCommitActionExecutor<>((HoodieJavaEngineContext) context, config, - this, instantTime, preppedRecords).execute(); + return convertMetadata(new InsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + new JavaCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_PREPPED)).execute()); } @Override @@ -124,24 +143,35 @@ public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineCont String instantTime, List> preppedRecords, Option>>> bulkInsertPartitioner) { - return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, - this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertPreppedCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + convertBulkInsertPartitioner(bulkInsertPartitioner), + new JavaCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT_PREPPED), + JavaBulkInsertHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> insertOverwrite(HoodieEngineContext context, String instantTime, List> records) { - return new JavaInsertOverwriteCommitActionExecutor( - context, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE, + HoodieListData.of(records), + new JavaInsertOverwriteCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE), + JavaWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, List> records) { - return new JavaInsertOverwriteTableCommitActionExecutor( - context, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, + HoodieListData.of(records), + new JavaInsertOverwriteTableCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE), + JavaWriteHelper.newInstance()).execute()); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java index a78b71b2402ba..2391f1845afa5 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java @@ -19,20 +19,24 @@ package org.apache.hudi.table; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.JavaUpsertPreppedDeltaCommitActionExecutor; +import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaBulkInsertHelper; +import org.apache.hudi.table.action.commit.JavaCommitHelper; +import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.JavaUpsertPreppedDeltaCommitHelper; import java.util.List; -public class HoodieJavaMergeOnReadTable extends HoodieJavaCopyOnWriteTable { +public class HoodieJavaMergeOnReadTable> extends HoodieJavaCopyOnWriteTable { protected HoodieJavaMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); } @@ -41,9 +45,9 @@ protected HoodieJavaMergeOnReadTable(HoodieWriteConfig config, HoodieEngineConte public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { - return new JavaUpsertPreppedDeltaCommitActionExecutor<>((HoodieJavaEngineContext) context, config, - this, instantTime, preppedRecords).execute(); - + return convertMetadata(new UpsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + new JavaUpsertPreppedDeltaCommitHelper<>(context, config, this, instantTime)).execute()); } @Override @@ -51,7 +55,10 @@ public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineCont String instantTime, List> preppedRecords, Option>>> bulkInsertPartitioner) { - return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, - this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertPreppedCommitActionExecutor( + context, config, this, instantTime, HoodieListData.of(preppedRecords), + convertBulkInsertPartitioner(bulkInsertPartitioner), + new JavaCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT_PREPPED), + JavaBulkInsertHelper.newInstance()).execute()); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java index 9cf9a6dc10cb0..130cf32fc47eb 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -28,12 +28,17 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.JavaHoodieIndex; +import org.apache.hudi.table.action.HoodieWriteMetadata; import java.util.List; +import static org.apache.hudi.table.action.commit.JavaCommitHelper.getList; + public abstract class HoodieJavaTable extends HoodieTable>, List, List> { protected HoodieJavaTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { @@ -61,6 +66,38 @@ public static HoodieJavaTable create(HoodieWr } } + public static Option>>> convertBulkInsertPartitioner( + Option>>> userDefinedBulkInsertPartitioner) { + Option>>> partitionerOption = Option.empty(); + if (userDefinedBulkInsertPartitioner.isPresent()) { + partitionerOption = Option.of( + convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner.get())); + } + return partitionerOption; + } + + public static BulkInsertPartitioner>> convertBulkInsertPartitioner( + BulkInsertPartitioner>> bulkInsertPartitioner) { + return new BulkInsertPartitioner>>() { + @Override + public HoodieData> repartitionRecords( + HoodieData> records, int outputSparkPartitions) { + return HoodieListData.of(bulkInsertPartitioner.repartitionRecords( + getList(records), outputSparkPartitions)); + } + + @Override + public boolean arePartitionRecordsSorted() { + return bulkInsertPartitioner.arePartitionRecordsSorted(); + } + }; + } + + public static HoodieWriteMetadata> convertMetadata( + HoodieWriteMetadata> metadata) { + return metadata.clone(getList(metadata.getWriteStatuses())); + } + @Override protected HoodieIndex>, List, List> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return JavaHoodieIndex.createIndex(config); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java deleted file mode 100644 index 9780262fb2b92..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java +++ /dev/null @@ -1,67 +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.client.common.HoodieJavaEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; -import java.util.Map; - -public class JavaBulkInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { - - private final List> inputRecords; - private final Option>>> bulkInsertPartitioner; - - public JavaBulkInsertCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, List> inputRecords, - Option>>> bulkInsertPartitioner) { - this(context, config, table, instantTime, inputRecords, bulkInsertPartitioner, Option.empty()); - } - - public JavaBulkInsertCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, List> inputRecords, - Option>>> bulkInsertPartitioner, - Option> extraMetadata) { - super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); - this.inputRecords = inputRecords; - this.bulkInsertPartitioner = bulkInsertPartitioner; - } - - @Override - public HoodieWriteMetadata> execute() { - try { - return JavaBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, table, config, - this, true, bulkInsertPartitioner); - } catch (HoodieInsertException ie) { - throw ie; - } catch (Throwable e) { - throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); - } - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java index b7ea916aafed8..cd39f2cd42f92 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java @@ -19,13 +19,14 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.execution.JavaLazyInsertIterable; import org.apache.hudi.execution.bulkinsert.JavaBulkInsertInternalPartitionerFactory; import org.apache.hudi.io.CreateHandleFactory; @@ -37,14 +38,16 @@ import java.util.ArrayList; import java.util.List; +import static org.apache.hudi.table.HoodieJavaTable.convertBulkInsertPartitioner; +import static org.apache.hudi.table.action.commit.JavaCommitHelper.getList; + /** - * A java implementation of {@link AbstractBulkInsertHelper}. + * A java implementation of {@link BaseBulkInsertHelper}. * * @param */ @SuppressWarnings("checkstyle:LineLength") -public class JavaBulkInsertHelper extends AbstractBulkInsertHelper>, - List, List, R> { +public class JavaBulkInsertHelper> extends BaseBulkInsertHelper { private JavaBulkInsertHelper() { } @@ -58,14 +61,12 @@ public static JavaBulkInsertHelper newInstance() { } @Override - public HoodieWriteMetadata> bulkInsert(final List> inputRecords, - final String instantTime, - final HoodieTable>, List, List> table, - final HoodieWriteConfig config, - final BaseCommitActionExecutor>, List, List, R> executor, - final boolean performDedupe, - final Option> userDefinedBulkInsertPartitioner) { - HoodieWriteMetadata result = new HoodieWriteMetadata(); + public HoodieWriteMetadata> bulkInsert( + final HoodieData> inputRecords, final String instantTime, final HoodieTable table, + final HoodieWriteConfig config, final boolean performDedupe, + final Option>>> userDefinedBulkInsertPartitioner, + final BaseCommitHelper commitHelper) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); // It's possible the transition to inflight could have already happened. if (!table.getActiveTimeline().filterInflights().containsInstant(instantTime)) { @@ -76,36 +77,36 @@ public HoodieWriteMetadata> bulkInsert(final List writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism(), false); + HoodieData writeStatuses = bulkInsert( + inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, + false, config.getBulkInsertShuffleParallelism(), false); //update index - ((BaseJavaCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatuses, result); + ((JavaCommitHelper) commitHelper).updateIndexAndCommitIfNeeded(getList(writeStatuses), result); return result; } @Override - public List bulkInsert(List> inputRecords, - String instantTime, - HoodieTable>, List, List> table, - HoodieWriteConfig config, - boolean performDedupe, - Option> userDefinedBulkInsertPartitioner, - boolean useWriterSchema, - int parallelism, - boolean preserveHoodieMetadata) { + public HoodieData bulkInsert( + HoodieData> inputRecords, String instantTime, HoodieTable table, + HoodieWriteConfig config, boolean performDedupe, + Option>>> userDefinedBulkInsertPartitioner, + boolean useWriterSchema, int parallelism, boolean preserveHoodieMetadata) { // De-dupe/merge if needed - List> dedupedRecords = inputRecords; + HoodieData> dedupedRecords = inputRecords; if (performDedupe) { - dedupedRecords = (List>) JavaWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, - parallelism, table); + dedupedRecords = (HoodieData>) JavaWriteHelper.newInstance().combineOnCondition( + config.shouldCombineBeforeInsert(), inputRecords, parallelism, table); } final List> repartitionedRecords; - BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() - ? userDefinedBulkInsertPartitioner.get() - : JavaBulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); - repartitionedRecords = (List>) partitioner.repartitionRecords(dedupedRecords, parallelism); + BulkInsertPartitioner>> partitioner = + userDefinedBulkInsertPartitioner.isPresent() + ? userDefinedBulkInsertPartitioner.get() + : convertBulkInsertPartitioner( + JavaBulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode())); + repartitionedRecords = getList(partitioner.repartitionRecords(dedupedRecords, parallelism)); FileIdPrefixProvider fileIdPrefixProvider = (FileIdPrefixProvider) ReflectionUtils.loadClass( config.getFileIdPrefixProviderClassName(), @@ -118,6 +119,6 @@ public List bulkInsert(List> inputRecords, fileIdPrefixProvider.createFilePrefix(""), table.getTaskContextSupplier(), new CreateHandleFactory<>()).forEachRemaining(writeStatuses::addAll); - return writeStatuses; + return HoodieListData.of(writeStatuses); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java deleted file mode 100644 index 37b56b6325bc3..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,63 +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.client.common.HoodieJavaEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.BulkInsertPartitioner; - -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class JavaBulkInsertPreppedCommitActionExecutor> - extends BaseJavaCommitActionExecutor { - - private final List> preppedInputRecord; - private final Option> userDefinedBulkInsertPartitioner; - - public JavaBulkInsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> preppedInputRecord, - Option> userDefinedBulkInsertPartitioner) { - super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); - this.preppedInputRecord = preppedInputRecord; - this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; - } - - @Override - public HoodieWriteMetadata> execute() { - try { - return JavaBulkInsertHelper.newInstance().bulkInsert(preppedInputRecord, instantTime, table, config, - this, false, userDefinedBulkInsertPartitioner); - } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } - throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); - } - } -} \ No newline at end of file diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaCommitHelper.java similarity index 54% rename from hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java rename to hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaCommitHelper.java index aec84a50e18af..2a9d9e8fd290c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaCommitHelper.java @@ -7,13 +7,14 @@ * "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 + * 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. + * 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; @@ -21,20 +22,18 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.JavaLazyInsertIterable; @@ -51,7 +50,6 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; import java.util.Collections; @@ -63,35 +61,40 @@ import java.util.Map; import java.util.stream.Collectors; -public abstract class BaseJavaCommitActionExecutor extends - BaseCommitActionExecutor>, List, List, HoodieWriteMetadata> { +public class JavaCommitHelper> extends BaseCommitHelper { - private static final Logger LOG = LogManager.getLogger(BaseJavaCommitActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(JavaCommitHelper.class); - public BaseJavaCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType) { - super(context, config, table, instantTime, operationType, Option.empty()); + public JavaCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + this(context, config, table, instantTime, operationType, Option.empty()); } - public BaseJavaCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType, - Option extraMetadata) { + public JavaCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { super(context, config, table, instantTime, operationType, extraMetadata); } + public static List getList(HoodieData hoodieData) { + return ((HoodieListData) hoodieData).get(); + } + + @Override + public void init(HoodieWriteConfig config) { + // No OP + } + @Override - public HoodieWriteMetadata> execute(List> inputRecords) { - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + public HoodieWriteMetadata> execute( + HoodieData> inputRecords) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); WorkloadProfile profile = null; if (isWorkloadProfileNeeded()) { - profile = new WorkloadProfile(buildProfile(inputRecords)); + profile = new WorkloadProfile(buildProfile(getList(inputRecords))); LOG.info("Workload profile :" + profile); try { saveWorkloadProfileMetadataToInflight(profile, instantTime); @@ -110,7 +113,7 @@ public HoodieWriteMetadata> execute(List> inpu } final Partitioner partitioner = getPartitioner(profile); - Map>> partitionedRecords = partition(inputRecords, partitioner); + Map>> partitionedRecords = partition(getList(inputRecords), partitioner); List writeStatuses = new LinkedList<>(); partitionedRecords.forEach((partition, records) -> { @@ -125,19 +128,16 @@ public HoodieWriteMetadata> execute(List> inpu return result; } - protected void updateIndex(List writeStatuses, HoodieWriteMetadata> result) { + protected void updateIndex( + List writeStatuses, HoodieWriteMetadata> result) { Instant indexStartTime = Instant.now(); // Update the index back - List statuses = table.getIndex().updateLocation(writeStatuses, context, table); + HoodieData statuses = HoodieListData.of((List) + table.getIndex().updateLocation(writeStatuses, context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); } - @Override - protected String getCommitActionType() { - return table.getMetaClient().getCommitActionType(); - } - private Partitioner getPartitioner(WorkloadProfile profile) { if (WriteOperationType.isChangingRecords(operationType)) { return getUpsertPartitioner(profile); @@ -146,17 +146,20 @@ private Partitioner getPartitioner(WorkloadProfile profile) { } } - private Map>> partition(List> dedupedRecords, Partitioner partitioner) { - Map>, HoodieRecord>>> partitionedMidRecords = dedupedRecords - .stream() - .map(record -> Pair.of(Pair.of(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)) - .collect(Collectors.groupingBy(x -> partitioner.getPartition(x.getLeft()))); + private Map>> partition( + List> dedupedRecords, Partitioner partitioner) { + Map>, HoodieRecord>>> partitionedMidRecords = + dedupedRecords.stream() + .map(record -> Pair.of(Pair.of( + record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)) + .collect(Collectors.groupingBy(x -> partitioner.getPartition(x.getLeft()))); Map>> results = new LinkedHashMap<>(); partitionedMidRecords.forEach((key, value) -> results.put(key, value.stream().map(x -> x.getRight()).collect(Collectors.toList()))); return results; } - protected Pair, WorkloadStat> buildProfile(List> inputRecords) { + protected Pair, WorkloadStat> buildProfile( + List> inputRecords) { HashMap partitionPathStatMap = new HashMap<>(); WorkloadStat globalStat = new WorkloadStat(); @@ -188,46 +191,13 @@ protected Pair, WorkloadStat> buildProfile(List> extraMetadata, HoodieWriteMetadata> result) { - commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList())); - } - - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { - String actionType = getCommitActionType(); - LOG.info("Committing " + instantTime + ", action Type " + actionType); - result.setCommitted(true); - result.setWriteStats(writeStats); - // Finalize write - finalizeWrite(instantTime, writeStats, result); - try { - LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), - extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); - - activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - LOG.info("Committed " + instantTime); - result.setCommitMetadata(Option.of(metadata)); - } catch (IOException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); - } - } - - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + protected Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeMetadata) { return Collections.emptyMap(); } - @Override - protected boolean isWorkloadProfileNeeded() { - return true; - } - - @SuppressWarnings("unchecked") - protected Iterator> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr, - Partitioner partitioner) { + protected Iterator> handleUpsertPartition( + String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) { JavaUpsertPartitioner javaUpsertPartitioner = (JavaUpsertPartitioner) partitioner; BucketInfo binfo = javaUpsertPartitioner.getBucketInfo(partition); BucketType btype = binfo.bucketType; @@ -246,65 +216,42 @@ protected Iterator> handleUpsertPartition(String instantTime, } } - protected Iterator> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, - Partitioner partitioner) { + protected Iterator> handleInsertPartition( + String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) { return handleUpsertPartition(instantTime, partition, recordItr, partitioner); } @Override - public Iterator> handleUpdate(String partitionPath, String fileId, - Iterator> recordItr) - throws IOException { + public Iterator> handleInsert( + String idPfx, Iterator> recordItr) { // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records if (!recordItr.hasNext()) { - LOG.info("Empty partition with fileId => " + fileId); + LOG.info("Empty partition"); return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); } - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); - return handleUpdateInternal(upsertHandle, fileId); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) - throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException( - "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); - } else { - JavaMergeHelper.newInstance().runMerge(table, upsertHandle); - } - - List statuses = upsertHandle.writeStatuses(); - if (upsertHandle.getPartitionPath() == null) { - LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + statuses); - } - return Collections.singletonList(statuses).iterator(); + return new JavaLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, + context.getTaskContextSupplier(), new CreateHandleFactory<>()); } - protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { + protected HoodieMergeHandle getUpdateHandle( + String partitionPath, String fileId, Iterator> recordItr) { if (table.requireSortedRecords()) { - return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier, Option.empty()); + return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, context.getTaskContextSupplier(), Option.empty()); } else { - return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier, Option.empty()); + return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, context.getTaskContextSupplier(), Option.empty()); } } - protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, - Map> keyToNewRecords, - HoodieBaseFile dataFileToBeMerged) { - return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords, - partitionPath, fileId, dataFileToBeMerged, taskContextSupplier, Option.empty()); + @Override + protected BaseMergeHelper getMergeHelper() { + return JavaMergeHelper.newInstance(); } - @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) { - // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records - if (!recordItr.hasNext()) { - LOG.info("Empty partition"); - return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); - } - return new JavaLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, - taskContextSupplier, new CreateHandleFactory<>()); + protected HoodieMergeHandle getUpdateHandle( + String partitionPath, String fileId, Map> keyToNewRecords, + HoodieBaseFile dataFileToBeMerged) { + return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords, + partitionPath, fileId, dataFileToBeMerged, context.getTaskContextSupplier(), Option.empty()); } /** @@ -324,13 +271,19 @@ public Partitioner getInsertPartitioner(WorkloadProfile profile) { return getUpsertPartitioner(profile); } - public void updateIndexAndCommitIfNeeded(List writeStatuses, HoodieWriteMetadata result) { + public void updateIndexAndCommitIfNeeded(List writeStatuses, HoodieWriteMetadata> result) { Instant indexStartTime = Instant.now(); // Update the index back - List statuses = table.getIndex().updateLocation(writeStatuses, context, table); + HoodieData statuses = HoodieListData.of((List) + table.getIndex().updateLocation(writeStatuses, context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); commitOnAutoCommit(result); } + + @Override + protected void syncTableMetadata() { + + } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java deleted file mode 100644 index 72c2332645cf3..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java +++ /dev/null @@ -1,46 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class JavaDeleteCommitActionExecutor> extends BaseJavaCommitActionExecutor { - private final List keys; - - public JavaDeleteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List keys) { - super(context, config, table, instantTime, WriteOperationType.DELETE); - this.keys = keys; - } - - @Override - public HoodieWriteMetadata> execute() { - return JavaDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java index a907c9fd9760e..fe9a47d25e0a0 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java @@ -23,8 +23,11 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -40,9 +43,7 @@ import java.util.List; import java.util.stream.Collectors; -@SuppressWarnings("checkstyle:LineLength") -public class JavaDeleteHelper extends - AbstractDeleteHelper>, List, List, R> { +public class JavaDeleteHelper> extends BaseDeleteHelper { private JavaDeleteHelper() { } @@ -56,37 +57,38 @@ public static JavaDeleteHelper newInstance() { } @Override - public List deduplicateKeys(List keys, - HoodieTable>, List, List> table, - int parallelism) { + public HoodieData deduplicateKeys( + HoodieData keys, + HoodieTable>, HoodieData, HoodieData> table, + int parallelism) { + List keysList = ((HoodieListData) keys).get(); boolean isIndexingGlobal = table.getIndex().isGlobal(); if (isIndexingGlobal) { - HashSet recordKeys = keys.stream().map(HoodieKey::getRecordKey).collect(Collectors.toCollection(HashSet::new)); + HashSet recordKeys = keysList.stream() + .map(HoodieKey::getRecordKey).collect(Collectors.toCollection(HashSet::new)); List deduplicatedKeys = new LinkedList<>(); - keys.forEach(x -> { + keysList.forEach(x -> { if (recordKeys.contains(x.getRecordKey())) { deduplicatedKeys.add(x); } }); - return deduplicatedKeys; + return HoodieListData.of(deduplicatedKeys); } else { - HashSet set = new HashSet<>(keys); - keys.clear(); - keys.addAll(set); + HashSet set = new HashSet<>(keysList); + keysList.clear(); + keysList.addAll(set); return keys; } } @Override - public HoodieWriteMetadata> execute(String instantTime, - List keys, - HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable>, List, List> table, - BaseCommitActionExecutor>, List, List, R> deleteExecutor) { + public HoodieWriteMetadata> execute( + String instantTime, HoodieData keys, HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + BaseCommitHelper commitHelper) { try { - HoodieWriteMetadata> result = null; - List dedupedKeys = keys; + HoodieWriteMetadata> result = null; + HoodieData dedupedKeys = keys; final int parallelism = config.getDeleteShuffleParallelism(); if (config.shouldCombineBeforeDelete()) { // De-dupe/merge if needed @@ -94,24 +96,28 @@ public HoodieWriteMetadata> execute(String instantTime, } List> dedupedRecords = - dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); + JavaCommitHelper.getList(dedupedKeys).stream().map(key -> + new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); Instant beginTag = Instant.now(); // perform index look up to get existing location of records - List> taggedRecords = - table.getIndex().tagLocation(dedupedRecords, context, table); + HoodieData> taggedRecords = (HoodieData>) + table.getIndex().tagLocation(HoodieListData.of(dedupedRecords), context, table); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records - List> taggedValidRecords = taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).collect(Collectors.toList()); + List> taggedValidRecords = + JavaCommitHelper.getList(taggedRecords).stream() + .filter(HoodieRecord::isCurrentLocationKnown).collect(Collectors.toList()); if (!taggedValidRecords.isEmpty()) { - result = deleteExecutor.execute(taggedValidRecords); + result = commitHelper.execute(HoodieListData.of(taggedValidRecords)); result.setIndexLookupDuration(tagLocationDuration); } else { // if entire set of keys are non existent - deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + commitHelper.saveWorkloadProfileMetadataToInflight( + new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); result = new HoodieWriteMetadata<>(); - result.setWriteStatuses(Collections.EMPTY_LIST); - deleteExecutor.commitOnAutoCommit(result); + result.setWriteStatuses(HoodieListData.of(Collections.EMPTY_LIST)); + commitHelper.commitOnAutoCommit(result); } return result; } catch (Throwable e) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java deleted file mode 100644 index 096aafcffc09c..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java +++ /dev/null @@ -1,50 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class JavaInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { - - private List> inputRecords; - - public JavaInsertCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - super(context, config, table, instantTime, WriteOperationType.INSERT); - this.inputRecords = inputRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java deleted file mode 100644 index b80191909351c..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java +++ /dev/null @@ -1,79 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class JavaInsertOverwriteCommitActionExecutor> - extends BaseJavaCommitActionExecutor { - - private final List> inputRecords; - - public JavaInsertOverwriteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> inputRecords) { - this(context, config, table, instantTime, inputRecords, WriteOperationType.INSERT_OVERWRITE); - } - - public JavaInsertOverwriteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> inputRecords, - WriteOperationType writeOperationType) { - super(context, config, table, instantTime, writeOperationType); - this.inputRecords = inputRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } - - @Override - protected String getCommitActionType() { - return HoodieTimeline.REPLACE_COMMIT_ACTION; - } - - @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeResult) { - return context.mapToPair( - writeResult.getWriteStatuses().stream().map(status -> status.getStat().getPartitionPath()).distinct().collect(Collectors.toList()), - partitionPath -> - Pair.of(partitionPath, getAllExistingFileIds(partitionPath)), 1 - ); - } - - private List getAllExistingFileIds(String partitionPath) { - // because new commit is not complete. it is safe to mark all existing file Ids as old files - return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitHelper.java new file mode 100644 index 0000000000000..ac4647d970746 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitHelper.java @@ -0,0 +1,63 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class JavaInsertOverwriteCommitHelper> extends JavaCommitHelper { + public JavaInsertOverwriteCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + super(context, config, table, instantTime, operationType); + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + @Override + public Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeResult) { + return context.mapToPair( + getList(writeResult.getWriteStatuses()).stream().map( + status -> status.getStat().getPartitionPath()).distinct().collect(Collectors.toList()), + partitionPath -> + Pair.of(partitionPath, getAllExistingFileIds(partitionPath)), 1); + } + + private List getAllExistingFileIds(String partitionPath) { + // because new commit is not complete. it is safe to mark all existing file Ids as old files + return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitHelper.java similarity index 60% rename from hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java rename to hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitHelper.java index 1170f2f4eac4c..59e8bfa71b229 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitHelper.java @@ -7,13 +7,14 @@ * "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 + * 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. + * 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; @@ -21,11 +22,12 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -34,22 +36,22 @@ import java.util.Map; import java.util.stream.Collectors; -public class JavaInsertOverwriteTableCommitActionExecutor> - extends JavaInsertOverwriteCommitActionExecutor { - - public JavaInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> inputRecords) { - super(context, config, table, instantTime, inputRecords, WriteOperationType.INSERT_OVERWRITE_TABLE); +public class JavaInsertOverwriteTableCommitHelper> + extends JavaInsertOverwriteCommitHelper { + public JavaInsertOverwriteTableCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + super(context, config, table, instantTime, operationType); } protected List getAllExistingFileIds(String partitionPath) { return table.getSliceView().getLatestFileSlices(partitionPath) - .map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); + .map(FileSlice::getFileId).distinct().collect(Collectors.toList()); } @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeResult) { + public Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeResult) { Map> partitionToExistingFileIds = new HashMap<>(); List partitionPaths = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getBasePath(), config.isMetadataTableEnabled(), diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java deleted file mode 100644 index 349cf69dcc30b..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.client.common.HoodieJavaEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class JavaInsertPreppedCommitActionExecutor> - extends BaseJavaCommitActionExecutor { - - private final List> preppedRecords; - - public JavaInsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> preppedRecords) { - super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java index a55121472310d..0df6d3a90cc50 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java @@ -44,7 +44,7 @@ import java.util.Iterator; import java.util.List; -public class JavaMergeHelper extends AbstractMergeHelper>, +public class JavaMergeHelper extends BaseMergeHelper>, List, List> { private JavaMergeHelper() { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java deleted file mode 100644 index 06fce78a36d32..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java +++ /dev/null @@ -1,50 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class JavaUpsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { - - private List> inputRecords; - - public JavaUpsertCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - List> inputRecords) { - super(context, config, table, instantTime, WriteOperationType.UPSERT); - this.inputRecords = inputRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java deleted file mode 100644 index 8eea5b5105826..0000000000000 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.client.common.HoodieJavaEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import java.util.List; - -public class JavaUpsertPreppedCommitActionExecutor> - extends BaseJavaCommitActionExecutor { - - private final List> preppedRecords; - - public JavaUpsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List> preppedRecords) { - super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java index ec7ea1641a442..088b21650f8b4 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java @@ -18,20 +18,22 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieTable; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; -public class JavaWriteHelper extends AbstractWriteHelper>, - List, List, R> { +public class JavaWriteHelper> extends BaseWriteHelper { private JavaWriteHelper() { } @@ -45,24 +47,33 @@ public static JavaWriteHelper newInstance() { } @Override - public List> deduplicateRecords(List> records, - HoodieIndex>, List, List> index, - int parallelism) { + protected HoodieData> tag( + HoodieData> dedupedRecords, HoodieEngineContext context, HoodieTable table) { + // perform index loop up to get existing location of records + return HoodieListData.of((List>) + table.getIndex().tagLocation(JavaCommitHelper.getList(dedupedRecords), context, table)); + } + + @Override + public HoodieData> deduplicateRecords( + HoodieData> records, HoodieIndex index, int parallelism) { boolean isIndexingGlobal = index.isGlobal(); - Map>>> keyedRecords = records.stream().map(record -> { - HoodieKey hoodieKey = record.getKey(); - // If index used is global, then records are expected to differ in their partitionPath - Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; - return Pair.of(key, record); - }).collect(Collectors.groupingBy(Pair::getLeft)); + Map>>> keyedRecords = + JavaCommitHelper.getList(records).stream().map(record -> { + HoodieKey hoodieKey = record.getKey(); + // If index used is global, then records are expected to differ in their partitionPath + Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; + return Pair.of(key, record); + }).collect(Collectors.groupingBy(Pair::getLeft)); - return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect - // everything - // so pick it from one of the records. - return new HoodieRecord(rec1.getKey(), reducedData); - }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); + return HoodieListData.of( + keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { + @SuppressWarnings("unchecked") + T reducedData = (T) rec1.getData().preCombine(rec2.getData()); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything + // so pick it from one of the records. + return new HoodieRecord(rec1.getKey(), reducedData); + }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList())); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitHelper.java similarity index 77% rename from hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java rename to hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitHelper.java index f6faa28bbb1ef..fc3ab2987f5d2 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitHelper.java @@ -20,18 +20,21 @@ package org.apache.hudi.table.action.deltacommit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.HoodieAppendHandle; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.JavaBulkInsertHelper; +import org.apache.hudi.table.action.commit.JavaCommitHelper; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -41,27 +44,25 @@ import java.util.LinkedList; import java.util.List; -public class JavaUpsertPreppedDeltaCommitActionExecutor> extends BaseJavaDeltaCommitActionExecutor { +public class JavaUpsertPreppedDeltaCommitHelper> extends JavaCommitHelper { - private static final Logger LOG = LogManager.getLogger(JavaUpsertPreppedDeltaCommitActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(JavaUpsertPreppedDeltaCommitHelper.class); - private final List> preppedInputRecords; - - public JavaUpsertPreppedDeltaCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, List> preppedInputRecords) { + public JavaUpsertPreppedDeltaCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); - this.preppedInputRecords = preppedInputRecords; } @Override - public HoodieWriteMetadata> execute() { - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + public HoodieWriteMetadata> execute( + HoodieData> preppedInputRecords) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); // First group by target file id. HashMap, List>> recordsByFileId = new HashMap<>(); List> insertedRecords = new LinkedList<>(); // Split records into inserts and updates. - for (HoodieRecord record : preppedInputRecords) { + for (HoodieRecord record : getList(preppedInputRecords)) { if (!record.isCurrentLocationKnown()) { insertedRecords.add(record); } else { @@ -79,14 +80,15 @@ public HoodieWriteMetadata> execute() { try { recordsByFileId.forEach((k, v) -> { HoodieAppendHandle appendHandle = new HoodieAppendHandle(config, instantTime, table, - k.getRight(), k.getLeft(), v.iterator(), taskContextSupplier); + k.getRight(), k.getLeft(), v.iterator(), context.getTaskContextSupplier()); appendHandle.doAppend(); allWriteStatuses.addAll(appendHandle.close()); }); if (insertedRecords.size() > 0) { HoodieWriteMetadata> insertResult = JavaBulkInsertHelper.newInstance() - .bulkInsert(insertedRecords, instantTime, table, config, this, false, Option.empty()); + .bulkInsert(HoodieListData.of(insertedRecords), instantTime, table, config, false, + Option.empty(), this); allWriteStatuses.addAll(insertResult.getWriteStatuses()); } } catch (Throwable e) { diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 796d7b74a83c5..b3fd9b7a81776 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -36,6 +37,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.io.HoodieCreateHandle; @@ -289,11 +291,11 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records - BaseJavaCommitActionExecutor actionExecutor = new JavaInsertCommitActionExecutor(context, config, table, - firstCommitTime, records); + JavaCommitHelper commitHelper = new JavaCommitHelper(context, config, table, + firstCommitTime, WriteOperationType.INSERT); List writeStatuses = new ArrayList<>(); - actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()) - .forEachRemaining(x -> writeStatuses.addAll((List)x)); + commitHelper.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()) + .forEachRemaining(x -> writeStatuses.addAll((List) x)); Map allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus.mergeMetadataForWriteStatuses(writeStatuses); @@ -332,12 +334,11 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs2 = records; - BaseJavaCommitActionExecutor actionExecutor = new JavaInsertPreppedCommitActionExecutor(context, config, table, - instantTime, recs2); + JavaCommitHelper commitHelper = new JavaCommitHelper(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); final List returnedStatuses = new ArrayList<>(); - actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator()) - .forEachRemaining(x -> returnedStatuses.addAll((List)x)); + commitHelper.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator()) + .forEachRemaining(x -> returnedStatuses.addAll((List) x)); assertEquals(2, returnedStatuses.size()); Map expectedPartitionNumRecords = new HashMap<>(); @@ -353,12 +354,12 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs3 = records; - BaseJavaCommitActionExecutor newActionExecutor = new JavaUpsertPreppedCommitActionExecutor(context, config, table, - instantTime, recs3); + JavaCommitHelper upsertPrepperCommitHelper = new JavaCommitHelper(context, config, table, + instantTime, WriteOperationType.UPSERT_PREPPED); final List returnedStatuses1 = new ArrayList<>(); - newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()) - .forEachRemaining(x -> returnedStatuses1.addAll((List)x)); + upsertPrepperCommitHelper.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()) + .forEachRemaining(x -> returnedStatuses1.addAll((List) x)); assertEquals(3, returnedStatuses1.size()); expectedPartitionNumRecords.clear(); @@ -388,11 +389,11 @@ public void testFileSizeUpsertRecords() throws Exception { } // Insert new records - BaseJavaCommitActionExecutor actionExecutor = new JavaUpsertCommitActionExecutor(context, config, table, - instantTime, records); + JavaCommitHelper commitHelper = new JavaCommitHelper(context, config, table, + instantTime, WriteOperationType.UPSERT); Arrays.asList(1).stream() - .map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) + .map(i -> commitHelper.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) .map(Transformations::flatten).collect(Collectors.toList()); // Check the updated file @@ -421,12 +422,12 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { String instantTime = "000"; // Perform inserts of 100 records to test CreateHandle and BufferedExecutor final List inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100); - BaseJavaCommitActionExecutor actionExecutor = new JavaInsertCommitActionExecutor(context, config, table, - instantTime, inserts); + JavaCommitHelper commitHelper = new JavaCommitHelper(context, config, table, + instantTime, WriteOperationType.INSERT); final List> ws = new ArrayList<>(); - actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator()) - .forEachRemaining(x -> ws.add((List)x)); + commitHelper.handleInsert(UUID.randomUUID().toString(), inserts.iterator()) + .forEachRemaining(x -> ws.add((List) x)); WriteStatus writeStatus = ws.get(0).get(0); String fileId = writeStatus.getFileId(); @@ -438,13 +439,13 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { String partitionPath = writeStatus.getPartitionPath(); long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); - BaseJavaCommitActionExecutor newActionExecutor = new JavaUpsertCommitActionExecutor(context, config, reloadedTable, - instantTime, updates); + JavaCommitHelper upsertCommitHelper = new JavaCommitHelper(context, config, reloadedTable, + instantTime, WriteOperationType.UPSERT); taskContextSupplier.reset(); final List> updateStatus = new ArrayList<>(); - newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator()) - .forEachRemaining(x -> updateStatus.add((List)x)); + upsertCommitHelper.handleUpdate(partitionPath, fileId, updates.iterator()) + .forEachRemaining(x -> updateStatus.add((List) x)); assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords()); } @@ -460,9 +461,11 @@ public void testBulkInsertRecords(String bulkInsertMode) throws Exception { // Insert new records final List inputRecords = generateTestRecordsForBulkInsert(); - JavaBulkInsertCommitActionExecutor bulkInsertExecutor = new JavaBulkInsertCommitActionExecutor( - context, config, table, instantTime, inputRecords, Option.empty()); - List returnedStatuses = (List)bulkInsertExecutor.execute().getWriteStatuses(); + BulkInsertCommitActionExecutor bulkInsertExecutor = new BulkInsertCommitActionExecutor( + context, config, table, instantTime, HoodieListData.of(inputRecords), Option.empty(), + new JavaCommitHelper(context, config, table, instantTime, WriteOperationType.BULK_INSERT), + JavaBulkInsertHelper.newInstance()); + List returnedStatuses = (List) bulkInsertExecutor.execute().getWriteStatuses(); verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords)); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/SparkHoodieRDDData.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/SparkHoodieRDDData.java new file mode 100644 index 0000000000000..e27969ed11376 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/SparkHoodieRDDData.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi; + +import org.apache.hudi.data.HoodieData; + +import org.apache.spark.api.java.JavaRDD; + +public class SparkHoodieRDDData extends HoodieData { + + private final JavaRDD rddData; + + private SparkHoodieRDDData(JavaRDD rddData) { + this.rddData = rddData; + } + + public static SparkHoodieRDDData of(JavaRDD rddData) { + return new SparkHoodieRDDData<>(rddData); + } + + @Override + public JavaRDD get() { + return rddData; + } + + @Override + public boolean isEmpty() { + return rddData.isEmpty(); + } +} 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 1df6eee903ca6..f7fff2dc24b62 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.SparkHoodieRDDData; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -29,6 +30,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; @@ -42,6 +44,7 @@ import java.util.Properties; import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; /** * Clustering Strategy based on following. @@ -69,16 +72,18 @@ public JavaRDD performClusteringWithRecordsRDD(final JavaRDD) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, - false, getPartitioner(strategyParams, schema), true, numOutputGroups, preserveHoodieMetadata); + return getRdd(SparkBulkInsertHelper.newInstance().bulkInsert( + SparkHoodieRDDData.of(inputRecords), instantTime, getHoodieTable(), newConfig, + false, HoodieSparkTable.convertBulkInsertPartitioner(getPartitioner(strategyParams, schema)), + true, numOutputGroups, preserveHoodieMetadata)); } /** * Create BulkInsertPartitioner based on strategy params. */ - protected Option> getPartitioner(Map strategyParams, Schema schema) { + protected Option>>> getPartitioner(Map strategyParams, Schema schema) { if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) { - return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","), + return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","), HoodieAvroUtils.addMetadataFields(schema))); } else { return Option.empty(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java index de06ea4b41148..28d0a1b6eed32 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java @@ -18,6 +18,7 @@ package org.apache.hudi.client.common; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.EngineProperty; @@ -28,8 +29,10 @@ import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieException; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; import scala.Tuple2; @@ -69,6 +72,15 @@ public static JavaSparkContext getSparkContext(HoodieEngineContext context) { return ((HoodieSparkEngineContext) context).getJavaSparkContext(); } + public static JavaRDD getRdd(HoodieData hoodieData) { + return ((SparkHoodieRDDData) hoodieData).get(); + } + + @Override + public List map(HoodieData data, SerializableFunction func) { + return getRdd(data).map(func::apply).collect(); + } + @Override public List map(List data, SerializableFunction func, int parallelism) { return javaSparkContext.parallelize(data, parallelism).map(func::apply).collect(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java index ab65e504e1f10..6b1f9874ab16d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java @@ -30,13 +30,13 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; + import scala.collection.JavaConverters; import java.util.Arrays; @@ -51,11 +51,11 @@ * Spark validator utils to verify and run any precommit validators configured. */ public class SparkValidatorUtils { - private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); + private static final Logger LOG = LogManager.getLogger(SparkValidatorUtils.class); /** * Check configured pre-commit validators and run them. Note that this only works for COW tables - * + *

* Throw error if there are validation failures. */ public static void runValidators(HoodieWriteConfig config, @@ -75,7 +75,7 @@ public static void runValidators(HoodieWriteConfig config, // Refresh timeline to ensure validator sees the any other operations done on timeline (async operations such as other clustering/compaction/rollback) table.getMetaClient().reloadActiveTimeline(); Dataset beforeState = getRecordsFromCommittedFiles(sqlContext, partitionsModified, table).cache(); - Dataset afterState = getRecordsFromPendingCommits(sqlContext, partitionsModified, writeMetadata, table, instantTime).cache(); + Dataset afterState = getRecordsFromPendingCommits(sqlContext, partitionsModified, writeMetadata, table, instantTime).cache(); Stream validators = Arrays.stream(config.getPreCommitValidators().split(",")) .map(validatorClass -> { @@ -141,8 +141,8 @@ public static Dataset readRecordsForBaseFiles(SQLContext sqlContext, List getRecordsFromPendingCommits(SQLContext sqlContext, - Set partitionsAffected, + public static Dataset getRecordsFromPendingCommits(SQLContext sqlContext, + Set partitionsAffected, HoodieWriteMetadata> writeMetadata, HoodieTable table, String instantTime) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index c2770a784780d..f4fa4b3a5a484 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -27,6 +28,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -34,6 +36,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -48,27 +51,38 @@ import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; -import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitHelper; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapHelper; import org.apache.hudi.table.action.clean.SparkCleanActionExecutor; import org.apache.hudi.table.action.clean.SparkCleanPlanActionExecutor; -import org.apache.hudi.table.action.cluster.SparkClusteringPlanActionExecutor; -import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkDeletePartitionCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; +import org.apache.hudi.table.action.cluster.SparkClusteringCommitHelper; +import org.apache.hudi.table.action.cluster.SparkClusteringHelper; +import org.apache.hudi.table.action.cluster.SparkClusteringPlanHelper; +import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.DeletePartitionCommitActionExecutor; +import org.apache.hudi.table.action.commit.ExecuteClusteringCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; +import org.apache.hudi.table.action.commit.SparkCommitHelper; +import org.apache.hudi.table.action.commit.SparkDeleteHelper; +import org.apache.hudi.table.action.commit.SparkDeletePartitionHelper; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitHelper; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitHelper; import org.apache.hudi.table.action.commit.SparkMergeHelper; -import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkWriteHelper; +import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -87,7 +101,7 @@ *

* UPDATES - Produce a new version of the file, just replacing the updated records with new values */ -public class HoodieSparkCopyOnWriteTable extends HoodieSparkTable { +public class HoodieSparkCopyOnWriteTable> extends HoodieSparkTable { private static final Logger LOG = LogManager.getLogger(HoodieSparkCopyOnWriteTable.class); @@ -97,58 +111,93 @@ public HoodieSparkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext @Override public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { - return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); + return convertMetadata(new UpsertCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(records), + new SparkCommitHelper<>(context, config, this, instantTime, WriteOperationType.UPSERT), + SparkWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { - return new SparkInsertCommitActionExecutor<>((HoodieSparkEngineContext)context, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT, SparkHoodieRDDData.of(records), + new SparkCommitHelper<>(context, config, this, instantTime, WriteOperationType.INSERT), + SparkWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD> records, Option>>> userDefinedBulkInsertPartitioner) { - return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, config, - this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(records), + convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner), + new SparkCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT_PREPPED), + SparkBulkInsertHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { - return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); + return convertMetadata(new DeleteCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(keys), + new SparkCommitHelper<>(context, config, this, instantTime, WriteOperationType.DELETE), + SparkDeleteHelper.newInstance()).execute()); } @Override - public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { - return new SparkDeletePartitionCommitActionExecutor(context, config, this, instantTime, partitions).execute(); + public HoodieWriteMetadata> deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + return convertMetadata(new DeletePartitionCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.DELETE_PARTITION, partitions, + new SparkInsertOverwriteCommitHelper<>( + context, config, this, instantTime, WriteOperationType.DELETE_PARTITION), + SparkDeletePartitionHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords) { - return new SparkUpsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new UpsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, SparkHoodieRDDData.of(preppedRecords), + new SparkCommitHelper<>( + context, config, this, instantTime, WriteOperationType.UPSERT_PREPPED)).execute()); } @Override public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords) { - return new SparkInsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new InsertPreppedCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(preppedRecords), + new SparkCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_PREPPED)).execute()); } @Override public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { - return new SparkBulkInsertPreppedCommitActionExecutor((HoodieSparkEngineContext) context, config, - this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertPreppedCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(preppedRecords), + convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner), + new SparkCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT_PREPPED), + SparkBulkInsertHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, JavaRDD> records) { - return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE, + SparkHoodieRDDData.of(records), + new SparkInsertOverwriteCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE), + SparkWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, JavaRDD> records) { - return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, + SparkHoodieRDDData.of(records), + new SparkInsertOverwriteTableCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE), + SparkWriteHelper.newInstance()).execute()); } @Override @@ -165,18 +214,29 @@ public HoodieWriteMetadata> compact(HoodieEngineContext con public Option scheduleClustering(HoodieEngineContext context, String instantTime, Option> extraMetadata) { - return new SparkClusteringPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute(); + return new ClusteringPlanActionExecutor<>( + context, config, this, instantTime, extraMetadata, SparkClusteringPlanHelper.newInstance()).execute(); } @Override public HoodieWriteMetadata> cluster(HoodieEngineContext context, String clusteringInstantTime) { - return new SparkExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute(); + return convertMetadata(new ExecuteClusteringCommitActionExecutor( + context, config, this, clusteringInstantTime, + new SparkClusteringCommitHelper<>((HoodieSparkEngineContext) context, config, this, clusteringInstantTime), + SparkClusteringHelper.newInstance()).execute()); } @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { - return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); + HoodieWriteConfig bootstrapConfig = new HoodieWriteConfig.Builder().withProps(config.getProps()) + .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) + .withBulkInsertParallelism(config.getBootstrapParallelism()) + .build(); + return convertBootstrapMetadata(new BootstrapCommitActionExecutor( + context, bootstrapConfig, this, extraMetadata, + new SparkBootstrapCommitHelper<>(context, bootstrapConfig, this, extraMetadata), + SparkBootstrapHelper.newInstance()).execute()); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java index ee66d7b0ab5b3..8aeb86ba5983d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -18,17 +18,20 @@ package org.apache.hudi.table; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -37,17 +40,24 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor; + +import org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitHelper; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaHelper; +import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; +import org.apache.hudi.table.action.commit.SparkDeleteHelper; +import org.apache.hudi.table.action.commit.SparkWriteHelper; +import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.compact.SparkRunCompactionActionExecutor; import org.apache.hudi.table.action.compact.SparkScheduleCompactionActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkBulkInsertPreppedDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkInsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkInsertPreppedDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkDeltaCommitHelper; import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor; import org.apache.hudi.table.action.restore.SparkMergeOnReadRestoreActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; @@ -74,7 +84,7 @@ * action *

*/ -public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable { +public class HoodieSparkMergeOnReadTable> extends HoodieSparkCopyOnWriteTable { HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); @@ -82,43 +92,64 @@ public class HoodieSparkMergeOnReadTable extends @Override public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { - return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); + return convertMetadata(new UpsertCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(records), + new SparkDeltaCommitHelper<>(context, config, this, instantTime, WriteOperationType.UPSERT), + SparkWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { - return new SparkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); + return convertMetadata(new InsertCommitActionExecutor( + context, config, this, instantTime, WriteOperationType.INSERT, SparkHoodieRDDData.of(records), + new SparkDeltaCommitHelper<>(context, config, this, instantTime, WriteOperationType.INSERT), + SparkWriteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD> records, Option>>> userDefinedBulkInsertPartitioner) { - return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, - this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(records), + convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner), + new SparkDeltaCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT_PREPPED), + SparkBulkInsertHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { - return new SparkDeleteDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); + return convertMetadata(new DeleteCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(keys), + new SparkDeltaCommitHelper<>(context, config, this, instantTime, WriteOperationType.DELETE), + SparkDeleteHelper.newInstance()).execute()); } @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords) { - return new SparkUpsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new UpsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, SparkHoodieRDDData.of(preppedRecords), + new SparkDeltaCommitHelper<>( + context, config, this, instantTime, WriteOperationType.UPSERT_PREPPED)).execute()); } @Override public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords) { - return new SparkInsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); + return convertMetadata(new InsertPreppedCommitActionExecutor<>( + context, config, this, instantTime, SparkHoodieRDDData.of(preppedRecords), + new SparkDeltaCommitHelper<>( + context, config, this, instantTime, WriteOperationType.INSERT_PREPPED)).execute()); } @Override public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { - return new SparkBulkInsertPreppedDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, - this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); + return convertMetadata(new BulkInsertPreppedCommitActionExecutor( + context, config, this, instantTime, SparkHoodieRDDData.of(preppedRecords), + convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner), + new SparkDeltaCommitHelper<>(context, config, this, instantTime, WriteOperationType.BULK_INSERT_PREPPED), + SparkBulkInsertHelper.newInstance()).execute()); } @Override @@ -136,7 +167,14 @@ public HoodieWriteMetadata> compact(HoodieEngineContext con @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { - return new SparkBootstrapDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); + HoodieWriteConfig bootstrapConfig = new HoodieWriteConfig.Builder().withProps(config.getProps()) + .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) + .withBulkInsertParallelism(config.getBootstrapParallelism()) + .build(); + return convertBootstrapMetadata(new BootstrapCommitActionExecutor( + context, bootstrapConfig, this, extraMetadata, + new SparkBootstrapCommitHelper<>(context, bootstrapConfig, this, extraMetadata), + SparkBootstrapDeltaHelper.newInstance()).execute()); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index 70a57b79e0f43..ad444f2551a3f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -28,12 +29,17 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.spark.api.java.JavaRDD; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; + public abstract class HoodieSparkTable extends HoodieTable>, JavaRDD, JavaRDD> { @@ -62,6 +68,58 @@ public static HoodieSparkTable create(HoodieW } } + public static Option>>> convertBulkInsertPartitioner( + Option>>> userDefinedBulkInsertPartitioner) { + Option>>> partitionerOption = Option.empty(); + if (userDefinedBulkInsertPartitioner.isPresent()) { + partitionerOption = Option.of(convertBulkInsertPartitioner(userDefinedBulkInsertPartitioner.get())); + } + return partitionerOption; + } + + public static BulkInsertPartitioner>> convertBulkInsertPartitioner( + BulkInsertPartitioner>> bulkInsertPartitioner) { + return new BulkInsertPartitioner>>() { + @Override + public HoodieData> repartitionRecords( + HoodieData> records, int outputSparkPartitions) { + return SparkHoodieRDDData.of(bulkInsertPartitioner.repartitionRecords( + getRdd(records), outputSparkPartitions)); + } + + @Override + public boolean arePartitionRecordsSorted() { + return bulkInsertPartitioner.arePartitionRecordsSorted(); + } + }; + } + + public static HoodieWriteMetadata> convertMetadata( + HoodieWriteMetadata> metadata) { + return metadata.clone(getRdd(metadata.getWriteStatuses())); + } + + public static HoodieBootstrapWriteMetadata> convertBootstrapMetadata( + HoodieBootstrapWriteMetadata> metadata) { + Option>> metadataBootstrapResult = + metadata.getMetadataBootstrapResult(); + Option>> fullBootstrapResult = + metadata.getFullBootstrapResult(); + Option>> newMetadataBootstrapResult = Option.empty(); + Option>> newFullBootstrapResult = Option.empty(); + if (metadataBootstrapResult.isPresent()) { + newMetadataBootstrapResult = Option.of(metadataBootstrapResult.get() + .clone(getRdd(metadataBootstrapResult.get().getWriteStatuses()))); + } + if (fullBootstrapResult.isPresent()) { + newFullBootstrapResult = Option.of(fullBootstrapResult.get() + .clone(getRdd(fullBootstrapResult.get().getWriteStatuses()))); + } + + return new HoodieBootstrapWriteMetadata<>( + newMetadataBootstrapResult, newFullBootstrapResult); + } + @Override protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return SparkHoodieIndex.createIndex(config); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitHelper.java new file mode 100644 index 0000000000000..188ff4d0fd0fe --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitHelper.java @@ -0,0 +1,200 @@ +/* + * 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.bootstrap; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; +import org.apache.hudi.client.utils.SparkValidatorUtils; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.BaseCommitHelper; +import org.apache.hudi.table.action.commit.BaseMergeHelper; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.table.HoodieSparkTable.convertMetadata; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; + +public class SparkBootstrapCommitHelper> extends BaseCommitHelper { + + private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitHelper.class); + protected String bootstrapSchema = null; + + public SparkBootstrapCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + Option> extraMetadata) { + super(context, config, table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + WriteOperationType.BOOTSTRAP, extraMetadata); + } + + @Override + public void init(HoodieWriteConfig config) { + } + + @Override + public HoodieWriteMetadata> execute( + HoodieData> inputRecords) { + // NO_OP + return null; + } + + public void setBootstrapSchema(String bootstrapSchema) { + this.bootstrapSchema = bootstrapSchema; + } + + @Override + public String getSchemaToStoreInCommit() { + return bootstrapSchema; + } + + @Override + public void commit( + Option> extraMetadata, + HoodieWriteMetadata> result) { + // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index + // is all done in a single job DAG. + Map>> bootstrapSourceAndStats = + getRdd(result.getWriteStatuses()).collect().stream() + .map(w -> { + BootstrapWriteStatus ws = (BootstrapWriteStatus) w; + return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); + }).collect(Collectors.groupingBy(w -> w.getKey().getPartitionPath())); + HoodieTableMetaClient metaClient = table.getMetaClient(); + try (BootstrapIndex.IndexWriter indexWriter = BootstrapIndex.getBootstrapIndex(metaClient) + .createWriter(metaClient.getTableConfig().getBootstrapBasePath().get())) { + LOG.info("Starting to write bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " + + config.getBasePath()); + indexWriter.begin(); + bootstrapSourceAndStats.forEach((key, value) -> indexWriter.appendNextPartition(key, + value.stream().map(Pair::getKey).collect(Collectors.toList()))); + indexWriter.finish(); + LOG.info("Finished writing bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " + + config.getBasePath()); + } + + commit(extraMetadata, result, bootstrapSourceAndStats.values().stream() + .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList())); + LOG.info("Committing metadata bootstrap !!"); + } + + protected void commit( + Option> extraMetadata, + HoodieWriteMetadata> result, List stats) { + String actionType = table.getMetaClient().getCommitActionType(); + LOG.info("Committing " + instantTime + ", action Type " + actionType); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieSparkTable table = HoodieSparkTable.create(config, context); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + + result.setCommitted(true); + stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); + result.setWriteStats(stats); + + // Finalize write + finalizeWrite(instantTime, stats, result); + syncTableMetadata(); + // add in extra metadata + if (extraMetadata.isPresent()) { + extraMetadata.get().forEach(metadata::addMetadata); + } + metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); + metadata.setOperationType(operationType); + + try { + activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + LOG.info("Committed " + instantTime); + } catch (IOException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, + e); + } + result.setCommitMetadata(Option.of(metadata)); + } + + @Override + public Iterator> handleInsert( + String idPfx, Iterator> recordItr) throws Exception { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + public Iterator> handleUpdate( + String partitionPath, String fileId, Iterator> recordItr) throws IOException { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + protected void syncTableMetadata() { + // Open up the metadata table again, for syncing + try (HoodieTableMetadataWriter writer = + SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context)) { + LOG.info("Successfully synced to metadata table"); + } catch (Exception e) { + throw new HoodieMetadataException("Error syncing to metadata table.", e); + } + } + + @Override + protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + protected BaseMergeHelper getMergeHelper() { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + SparkValidatorUtils.runValidators(config, convertMetadata(writeMetadata), context, table, instantTime); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java deleted file mode 100644 index 59f86662b7c0c..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.bootstrap; - -import java.util.Map; - -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor; -import org.apache.spark.api.java.JavaRDD; - -public class SparkBootstrapDeltaCommitActionExecutor> - extends SparkBootstrapCommitActionExecutor { - - public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - Option> extraMetadata) { - super(context, config, table, extraMetadata); - } - - @Override - protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { - return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) - .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, - inputRecordsRDD, extraMetadata); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaHelper.java new file mode 100644 index 0000000000000..2efa4060a319b --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaHelper.java @@ -0,0 +1,68 @@ +/* + * 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.bootstrap; + +import org.apache.hudi.SparkHoodieRDDData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; +import org.apache.hudi.table.action.deltacommit.SparkDeltaCommitHelper; + +import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; + +public class SparkBootstrapDeltaHelper> + extends SparkBootstrapHelper { + + private SparkBootstrapDeltaHelper() { + } + + private static class BootstrapDeltaHelperHolder { + private static final SparkBootstrapDeltaHelper SPARK_BOOTSTRAP_DELTA_HELPER = new SparkBootstrapDeltaHelper(); + } + + public static SparkBootstrapDeltaHelper newInstance() { + return SparkBootstrapDeltaHelper.BootstrapDeltaHelperHolder.SPARK_BOOTSTRAP_DELTA_HELPER; + } + + @Override + protected BulkInsertCommitActionExecutor getBulkInsertActionExecutor( + HoodieEngineContext context, HoodieTable table, HoodieWriteConfig config, + Option> extraMetadata, String bootstrapSchema, + JavaRDD> inputRecordsRDD) { + HoodieWriteConfig bulkInsertConfig = new HoodieWriteConfig.Builder() + .withProps(config.getProps()).withSchema(bootstrapSchema).build(); + return new BulkInsertCommitActionExecutor( + context, bulkInsertConfig, + table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + SparkHoodieRDDData.of(inputRecordsRDD), Option.empty(), extraMetadata, + new SparkDeltaCommitHelper<>(context, bulkInsertConfig, table, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, WriteOperationType.BULK_INSERT), + SparkBulkInsertHelper.newInstance()); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapHelper.java similarity index 54% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapHelper.java index 821b3071e145c..438785e048fa8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapHelper.java @@ -7,46 +7,43 @@ * "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 + * 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. + * 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.bootstrap; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.HoodieSparkBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; import org.apache.hudi.client.bootstrap.translator.BootstrapPartitionPathTranslator; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.client.utils.SparkValidatorUtils; import org.apache.hudi.common.bootstrap.FileStatusUtils; -import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BootstrapFileMapping; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetReaderIterator; @@ -55,23 +52,20 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieKeyGeneratorException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieBootstrapHandle; import org.apache.hudi.keygen.KeyGeneratorInterface; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; -import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -import org.apache.hudi.table.action.commit.BaseCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.BaseCommitHelper; +import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; +import org.apache.hudi.table.action.commit.SparkCommitHelper; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -93,44 +87,34 @@ import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class SparkBootstrapCommitActionExecutor> - extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { +public class SparkBootstrapHelper> extends BaseBootstrapHelper { - private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class); - protected String bootstrapSchema = null; - private transient FileSystem bootstrapSourceFileSystem; + private static final Logger LOG = LogManager.getLogger(SparkBootstrapHelper.class); - public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, - HoodieTable>, JavaRDD, JavaRDD> table, - Option> extraMetadata) { - super(context, new HoodieWriteConfig.Builder().withProps(config.getProps()) - .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) - .withBulkInsertParallelism(config.getBootstrapParallelism()) - .build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP, - extraMetadata); - bootstrapSourceFileSystem = FSUtils.getFs(config.getBootstrapSourceBasePath(), hadoopConf); + protected SparkBootstrapHelper() { } - private void validate() { - ValidationUtils.checkArgument(config.getBootstrapSourceBasePath() != null, - "Ensure Bootstrap Source Path is set"); - ValidationUtils.checkArgument(config.getBootstrapModeSelectorClass() != null, - "Ensure Bootstrap Partition Selector is set"); + private static class BootstrapHelperHolder { + private static final SparkBootstrapHelper SPARK_BOOTSTRAP_HELPER = new SparkBootstrapHelper(); + } + + public static SparkBootstrapHelper newInstance() { + return BootstrapHelperHolder.SPARK_BOOTSTRAP_HELPER; } @Override - public HoodieBootstrapWriteMetadata execute() { - validate(); + public HoodieBootstrapWriteMetadata> execute( + HoodieEngineContext context, HoodieTable table, HoodieWriteConfig config, + Option> extraMetadata, + BaseCommitHelper commitHelper) { + validate(config); try { HoodieTableMetaClient metaClient = table.getMetaClient(); Option completetedInstant = @@ -138,144 +122,90 @@ public HoodieBootstrapWriteMetadata execute() { ValidationUtils.checkArgument(!completetedInstant.isPresent(), "Active Timeline is expected to be empty for bootstrap to be performed. " + "If you want to re-bootstrap, please rollback bootstrap first !!"); - Map>>> partitionSelections = listAndProcessSourcePartitions(); + Map>>> partitionSelections = + listAndProcessSourcePartitions(context, table, config, (SparkBootstrapCommitHelper) commitHelper); // First run metadata bootstrap which will auto commit - Option metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); + Option>> metadataResult = + metadataBootstrap(context, table, config, commitHelper, partitionSelections.get(BootstrapMode.METADATA_ONLY)); // if there are full bootstrap to be performed, perform that too - Option fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); - return new HoodieBootstrapWriteMetadata(metadataResult, fullBootstrapResult); + Option>> fullBootstrapResult = + fullBootstrap(context, table, config, extraMetadata, commitHelper.getSchemaToStoreInCommit(), + partitionSelections.get(BootstrapMode.FULL_RECORD)); + return new HoodieBootstrapWriteMetadata<>(metadataResult, fullBootstrapResult); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } } - @Override - protected String getSchemaToStoreInCommit() { - return bootstrapSchema; - } - /** * Perform Metadata Bootstrap. + * * @param partitionFilesList List of partitions and files within that partitions */ - protected Option metadataBootstrap(List>> partitionFilesList) { + protected Option>> metadataBootstrap( + HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config, + BaseCommitHelper commitHelper, + List>> partitionFilesList) { if (null == partitionFilesList || partitionFilesList.isEmpty()) { return Option.empty(); } HoodieTableMetaClient metaClient = table.getMetaClient(); metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(), + new HoodieInstant(HoodieInstant.State.REQUESTED, metaClient.getCommitActionType(), HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)); - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, metaClient.getCommitActionType(), HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS), Option.empty()); - JavaRDD bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); + JavaRDD bootstrapWriteStatuses = + runMetadataBootstrap(context, table, config, partitionFilesList); - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); - updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + updateIndexAndCommitIfNeeded(context, table, config, commitHelper, bootstrapWriteStatuses.map(w -> w), result); return Option.of(result); } - private void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata> result) { + private void updateIndexAndCommitIfNeeded(HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config, + BaseCommitHelper commitHelper, + JavaRDD writeStatusRDD, + HoodieWriteMetadata> result) { // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future // RDD actions that are performed after updating the index. writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = table.getIndex().updateLocation(writeStatusRDD, context, + JavaRDD statuses = (JavaRDD) table.getIndex().updateLocation(writeStatusRDD, context, table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); - result.setWriteStatuses(statuses); - commitOnAutoCommit(result); - } - - @Override - public HoodieWriteMetadata> execute(JavaRDD> inputRecords) { - // NO_OP - return null; + result.setWriteStatuses(SparkHoodieRDDData.of(statuses)); + commitHelper.commitOnAutoCommit(result); } - @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { - // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index - // is all done in a single job DAG. - Map>> bootstrapSourceAndStats = - result.getWriteStatuses().collect().stream() - .map(w -> { - BootstrapWriteStatus ws = (BootstrapWriteStatus) w; - return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); - }).collect(Collectors.groupingBy(w -> w.getKey().getPartitionPath())); - HoodieTableMetaClient metaClient = table.getMetaClient(); - try (BootstrapIndex.IndexWriter indexWriter = BootstrapIndex.getBootstrapIndex(metaClient) - .createWriter(metaClient.getTableConfig().getBootstrapBasePath().get())) { - LOG.info("Starting to write bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " - + config.getBasePath()); - indexWriter.begin(); - bootstrapSourceAndStats.forEach((key, value) -> indexWriter.appendNextPartition(key, - value.stream().map(Pair::getKey).collect(Collectors.toList()))); - indexWriter.finish(); - LOG.info("Finished writing bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " - + config.getBasePath()); - } - - commit(extraMetadata, result, bootstrapSourceAndStats.values().stream() - .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList())); - LOG.info("Committing metadata bootstrap !!"); - } - - @Override - protected void syncTableMetadata() { - // Open up the metadata table again, for syncing - try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { - LOG.info("Successfully synced to metadata table"); - } catch (Exception e) { - throw new HoodieMetadataException("Error syncing to metadata table.", e); - } - } - - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { - String actionType = table.getMetaClient().getCommitActionType(); - LOG.info("Committing " + instantTime + ", action Type " + actionType); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieSparkTable table = HoodieSparkTable.create(config, context); - - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - - result.setCommitted(true); - stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); - result.setWriteStats(stats); - - // Finalize write - finalizeWrite(instantTime, stats, result); - syncTableMetadata(); - // add in extra metadata - if (extraMetadata.isPresent()) { - extraMetadata.get().forEach(metadata::addMetadata); - } - metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); - metadata.setOperationType(operationType); - - try { - activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - LOG.info("Committed " + instantTime); - } catch (IOException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); - } - result.setCommitMetadata(Option.of(metadata)); + private void validate(HoodieWriteConfig config) { + ValidationUtils.checkArgument(config.getBootstrapSourceBasePath() != null, + "Ensure Bootstrap Source Path is set"); + ValidationUtils.checkArgument(config.getBootstrapModeSelectorClass() != null, + "Ensure Bootstrap Partition Selector is set"); } /** * Perform Full Bootstrap. + * * @param partitionFilesList List of partitions and files within that partitions */ - protected Option fullBootstrap(List>> partitionFilesList) { + protected Option>> fullBootstrap( + HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config, + Option> extraMetadata, + String bootstrapSchema, + List>> partitionFilesList) { if (null == partitionFilesList || partitionFilesList.isEmpty()) { return Option.empty(); } @@ -284,29 +214,42 @@ protected Option fullBootstrap(List inputRecordsRDD = - (JavaRDD) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(), + JavaRDD> inputRecordsRDD = + (JavaRDD>) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(), partitionFilesList); // Start Full Bootstrap - final HoodieInstant requested = new HoodieInstant(State.REQUESTED, table.getMetaClient().getCommitActionType(), + final HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, table.getMetaClient().getCommitActionType(), HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); table.getActiveTimeline().createNewInstant(requested); // Setup correct schema and run bulk insert. - return Option.of(getBulkInsertActionExecutor(inputRecordsRDD).execute()); + return Option.of(getBulkInsertActionExecutor(context, table, config, extraMetadata, bootstrapSchema, inputRecordsRDD).execute()); } - protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { - return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) - .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, - inputRecordsRDD, Option.empty(), extraMetadata); + protected BulkInsertCommitActionExecutor getBulkInsertActionExecutor( + HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config, + Option> extraMetadata, + String bootstrapSchema, + JavaRDD> inputRecordsRDD) { + HoodieWriteConfig bulkInsertConfig = new HoodieWriteConfig.Builder() + .withProps(config.getProps()).withSchema(bootstrapSchema).build(); + return new BulkInsertCommitActionExecutor( + context, bulkInsertConfig, + table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + SparkHoodieRDDData.of(inputRecordsRDD), Option.empty(), extraMetadata, + new SparkCommitHelper<>((HoodieSparkEngineContext) context, bulkInsertConfig, table, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, WriteOperationType.BULK_INSERT), + SparkBulkInsertHelper.newInstance()); } - private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, String partitionPath, + private BootstrapWriteStatus handleMetadataBootstrap(HoodieTable table, HoodieWriteConfig config, + String srcPartitionPath, String partitionPath, HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) { Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); - HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier()); Schema avroSchema = null; try { @@ -322,7 +265,7 @@ private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, St BoundedInMemoryExecutor wrapper = null; try (ParquetReader reader = - AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build()) { + AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build()) { wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> { String recKey = keyGenerator.getKey(inp).getRecordKey(); @@ -355,18 +298,27 @@ private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, St /** * Return Bootstrap Mode selections for partitions listed and figure out bootstrap Schema. + * * @return * @throws IOException */ - private Map>>> listAndProcessSourcePartitions() throws IOException { + private Map>>> listAndProcessSourcePartitions( + HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config, + SparkBootstrapCommitHelper commitHelper) throws IOException { + FileSystem bootstrapSourceFileSystem = + FSUtils.getFs(config.getBootstrapSourceBasePath(), context.getHadoopConf().get()); List>> folders = BootstrapUtils.getAllLeafFoldersWithFiles( - table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), context); + table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), context); LOG.info("Fetching Bootstrap Schema !!"); HoodieBootstrapSchemaProvider sourceSchemaProvider = new HoodieSparkBootstrapSchemaProvider(config); - bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(context, folders).toString(); + String bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(context, folders).toString(); + commitHelper.setBootstrapSchema(bootstrapSchema); LOG.info("Bootstrap Schema :" + bootstrapSchema); + BootstrapModeSelector selector = (BootstrapModeSelector) ReflectionUtils.loadClass(config.getBootstrapModeSelectorClass(), config); @@ -383,7 +335,11 @@ private Map>>> listAndPr .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } - private JavaRDD runMetadataBootstrap(List>> partitions) { + private JavaRDD runMetadataBootstrap( + HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config, + List>> partitions) { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); if (null == partitions || partitions.isEmpty()) { return jsc.emptyRDD(); @@ -410,22 +366,7 @@ private JavaRDD runMetadataBootstrap(List handleMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), + .map(partitionFsPair -> handleMetadataBootstrap(table, config, partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), partitionFsPair.getRight().getRight(), keyGenerator)); } - - @Override - protected Iterator> handleInsert(String idPfx, Iterator> recordItr) { - throw new UnsupportedOperationException("Should not called in bootstrap code path"); - } - - @Override - protected Iterator> handleUpdate(String partitionPath, String fileId, Iterator> recordItr) { - throw new UnsupportedOperationException("Should not called in bootstrap code path"); - } - - @Override - protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { - SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringCommitHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringCommitHelper.java new file mode 100644 index 0000000000000..92077a0bb4d9d --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringCommitHelper.java @@ -0,0 +1,77 @@ +/* + * 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.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.SparkCommitHelper; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class SparkClusteringCommitHelper> extends SparkCommitHelper { + + private final HoodieClusteringPlan clusteringPlan; + + public SparkClusteringCommitHelper( + HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime) { + super(context, config, table, instantTime, WriteOperationType.CLUSTER); + this.clusteringPlan = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime)) + .map(Pair::getRight).orElseThrow( + () -> new HoodieClusteringException("Unable to read clustering plan for instant: " + instantTime)); + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + @Override + public Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeMetadata) { + Set newFilesWritten = new HashSet(writeMetadata.getWriteStats().get().stream() + .map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())) + .collect(Collectors.toList())); + return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan) + .filter(fg -> !newFilesWritten.contains(fg)) + .collect(Collectors.groupingBy(fg -> fg.getPartitionPath(), Collectors.mapping(fg -> fg.getFileId(), Collectors.toList()))); + } + + public HoodieClusteringPlan getClusteringPlan() { + return clusteringPlan; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringHelper.java similarity index 50% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringHelper.java index e734b4a406a67..e510ddd62ee4a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringHelper.java @@ -7,88 +7,88 @@ * "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 + * 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. + * 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.cluster; +import org.apache.hudi.SparkHoodieRDDData; 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.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; -import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; +import org.apache.hudi.table.action.commit.BaseClusteringHelper; +import org.apache.hudi.table.action.commit.BaseCommitHelper; import org.apache.avro.Schema; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; -import java.util.HashSet; -import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -public class SparkExecuteClusteringCommitActionExecutor> - extends BaseSparkCommitActionExecutor { +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; - private static final Logger LOG = LogManager.getLogger(SparkExecuteClusteringCommitActionExecutor.class); - private final HoodieClusteringPlan clusteringPlan; +public class SparkClusteringHelper> extends BaseClusteringHelper { + private SparkClusteringHelper() { + } + + private static class ClusteringHelperHolder { + private static final SparkClusteringHelper SPARK_CLUSTERING_HELPER = new SparkClusteringHelper(); + } - public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime) { - super(context, config, table, instantTime, WriteOperationType.CLUSTER); - this.clusteringPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime)) - .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException("Unable to read clustering plan for instant: " + instantTime)); + public static SparkClusteringHelper newInstance() { + return ClusteringHelperHolder.SPARK_CLUSTERING_HELPER; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute( + HoodieEngineContext context, String instantTime, HoodieTable table, HoodieWriteConfig config, + Option> extraMetadata, + BaseCommitHelper commitHelper) { + SparkClusteringCommitHelper sparkClusteringCommitHelper = (SparkClusteringCommitHelper) commitHelper; HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime); // Mark instant as clustering inflight table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); table.getMetaClient().reloadActiveTimeline(); final Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); - HoodieWriteMetadata> writeMetadata = ((ClusteringExecutionStrategy>, JavaRDD, JavaRDD>) + HoodieWriteMetadata> writeMetaDataRdd = ((ClusteringExecutionStrategy>, JavaRDD, JavaRDD>) ReflectionUtils.loadClass(config.getClusteringExecutionStrategyClass(), new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config)) - .performClustering(clusteringPlan, schema, instantTime); - JavaRDD writeStatusRDD = writeMetadata.getWriteStatuses(); - JavaRDD statuses = updateIndex(writeStatusRDD, writeMetadata); - writeMetadata.setWriteStats(statuses.map(WriteStatus::getStat).collect()); - writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeMetadata)); - validateWriteResult(writeMetadata); - commitOnAutoCommit(writeMetadata); + .performClustering(sparkClusteringCommitHelper.getClusteringPlan(), schema, instantTime); + HoodieWriteMetadata> writeMetadata = writeMetaDataRdd.clone(SparkHoodieRDDData.of(writeMetaDataRdd.getWriteStatuses())); + JavaRDD writeStatusRDD = writeMetaDataRdd.getWriteStatuses(); + HoodieData statuses = sparkClusteringCommitHelper.updateIndex(writeStatusRDD, writeMetadata); + writeMetadata.setWriteStats(getRdd(statuses).map(WriteStatus::getStat).collect()); + writeMetadata.setPartitionToReplaceFileIds(sparkClusteringCommitHelper.getPartitionToReplacedFileIds(writeMetadata)); + validateWriteResult(writeMetadata, instantTime, sparkClusteringCommitHelper.getClusteringPlan()); + sparkClusteringCommitHelper.commitOnAutoCommit(writeMetadata); if (!writeMetadata.getCommitMetadata().isPresent()) { HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeMetadata.getWriteStats().get(), writeMetadata.getPartitionToReplaceFileIds(), - extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + extraMetadata, WriteOperationType.CLUSTER, sparkClusteringCommitHelper.getSchemaToStoreInCommit(), sparkClusteringCommitHelper.getCommitActionType()); writeMetadata.setCommitMetadata(Option.of(commitMetadata)); } return writeMetadata; @@ -99,7 +99,9 @@ public HoodieWriteMetadata> execute() { * But we can extend this to add more validation. E.g. number of records read = number of records written etc. * We can also make these validations in BaseCommitActionExecutor to reuse pre-commit hooks for multiple actions. */ - private void validateWriteResult(HoodieWriteMetadata> writeMetadata) { + private void validateWriteResult( + HoodieWriteMetadata> writeMetadata, String instantTime, + HoodieClusteringPlan clusteringPlan) { if (writeMetadata.getWriteStatuses().isEmpty()) { throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime + " #groups: " + clusteringPlan.getInputGroups().size() + " expected at least " @@ -107,19 +109,4 @@ private void validateWriteResult(HoodieWriteMetadata> write + " write statuses"); } } - - @Override - protected String getCommitActionType() { - return HoodieTimeline.REPLACE_COMMIT_ACTION; - } - - @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - Set newFilesWritten = new HashSet(writeMetadata.getWriteStats().get().stream() - .map(s -> new HoodieFileGroupId(s.getPartitionPath(),s.getFileId())) - .collect(Collectors.toList())); - return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan) - .filter(fg -> !newFilesWritten.contains(fg)) - .collect(Collectors.groupingBy(fg -> fg.getPartitionPath(), Collectors.mapping(fg -> fg.getFileId(), Collectors.toList()))); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanHelper.java similarity index 65% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanHelper.java index 683d852131f6d..f638dc369c7f7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanHelper.java @@ -7,22 +7,20 @@ * "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 + * 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. + * 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.cluster; import org.apache.hudi.avro.model.HoodieClusteringPlan; -import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -30,28 +28,29 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; -import java.util.Map; +public class SparkClusteringPlanHelper> + extends BaseClusteringPlanHelper { -@SuppressWarnings("checkstyle:LineLength") -public class SparkClusteringPlanActionExecutor extends - BaseClusteringPlanActionExecutor>, JavaRDD, JavaRDD> { + private static final Logger LOG = LogManager.getLogger(SparkClusteringPlanHelper.class); - private static final Logger LOG = LogManager.getLogger(SparkClusteringPlanActionExecutor.class); + private SparkClusteringPlanHelper() { + } - public SparkClusteringPlanActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable>, JavaRDD, JavaRDD> table, - String instantTime, - Option> extraMetadata) { - super(context, config, table, instantTime, extraMetadata); + private static class ClusteringPlanHelperHolder { + private static final SparkClusteringPlanHelper SPARK_CLUSTERING_PLAN_HELPER = new SparkClusteringPlanHelper(); + } + + public static SparkClusteringPlanHelper newInstance() { + return ClusteringPlanHelperHolder.SPARK_CLUSTERING_PLAN_HELPER; } @Override - protected Option createClusteringPlan() { + public Option createClusteringPlan( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table) { LOG.info("Checking if clustering needs to be run on " + config.getBasePath()); Option lastClusteringInstant = table.getActiveTimeline().getCompletedReplaceTimeline().lastInstant(); @@ -77,5 +76,4 @@ protected Option createClusteringPlan() { ReflectionUtils.loadClass(config.getClusteringPlanStrategyClass(), table, context, config); return strategy.generateClusteringPlan(); } - } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java deleted file mode 100644 index 2b00d47b01564..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java +++ /dev/null @@ -1,67 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -import java.util.Map; - -public class SparkBulkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { - - private final JavaRDD> inputRecordsRDD; - private final Option> bulkInsertPartitioner; - - public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner) { - this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); - } - - public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner, - Option> extraMetadata) { - super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); - this.inputRecordsRDD = inputRecordsRDD; - this.bulkInsertPartitioner = bulkInsertPartitioner; - } - - @Override - public HoodieWriteMetadata> execute() { - try { - return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, - this, true, bulkInsertPartitioner); - } catch (HoodieInsertException ie) { - throw ie; - } catch (Throwable e) { - throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 322d19194ae81..b77ae9c5fb2d1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -18,33 +18,37 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory; import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.api.java.JavaRDD; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.hudi.table.HoodieSparkTable.convertBulkInsertPartitioner; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; + /** - * A spark implementation of {@link AbstractBulkInsertHelper}. + * A spark implementation of {@link BaseBulkInsertHelper}. * * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkBulkInsertHelper extends AbstractBulkInsertHelper>, - JavaRDD, JavaRDD, R> { +public class SparkBulkInsertHelper> extends BaseBulkInsertHelper { private SparkBulkInsertHelper() { } @@ -58,60 +62,59 @@ public static SparkBulkInsertHelper newInstance() { } @Override - public HoodieWriteMetadata> bulkInsert(final JavaRDD> inputRecords, - final String instantTime, - final HoodieTable>, JavaRDD, JavaRDD> table, - final HoodieWriteConfig config, - final BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, - final boolean performDedupe, - final Option> userDefinedBulkInsertPartitioner) { - HoodieWriteMetadata result = new HoodieWriteMetadata(); + public HoodieWriteMetadata> bulkInsert( + final HoodieData> inputRecords, final String instantTime, final HoodieTable table, + final HoodieWriteConfig config, final boolean performDedupe, + final Option>>> userDefinedBulkInsertPartitioner, + final BaseCommitHelper commitHelper) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); //transition bulk_insert state to inflight table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), config.shouldAllowMultiWriteOnSameInstant()); // write new files - JavaRDD writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism(), false); + HoodieData writeStatuses = + bulkInsert(inputRecords, instantTime, table, config, performDedupe, + userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism(), false); //update index - ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatuses, result); + ((SparkCommitHelper) commitHelper).updateIndexAndCommitIfNeeded(getRdd(writeStatuses), result); return result; } @Override - public JavaRDD bulkInsert(JavaRDD> inputRecords, - String instantTime, - HoodieTable>, JavaRDD, JavaRDD> table, - HoodieWriteConfig config, - boolean performDedupe, - Option> userDefinedBulkInsertPartitioner, - boolean useWriterSchema, - int parallelism, - boolean preserveMetadata) { + public HoodieData bulkInsert( + HoodieData> inputRecords, String instantTime, HoodieTable table, + HoodieWriteConfig config, boolean performDedupe, + Option>>> userDefinedBulkInsertPartitioner, + boolean useWriterSchema, int parallelism, boolean preserveMetadata) { // De-dupe/merge if needed - JavaRDD> dedupedRecords = inputRecords; + HoodieData> dedupedRecords = inputRecords; if (performDedupe) { - dedupedRecords = (JavaRDD>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, - parallelism, table); + dedupedRecords = SparkWriteHelper.newInstance().combineOnCondition( + config.shouldCombineBeforeInsert(), inputRecords, parallelism, table); } final JavaRDD> repartitionedRecords; - BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() - ? userDefinedBulkInsertPartitioner.get() - : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); - repartitionedRecords = (JavaRDD>) partitioner.repartitionRecords(dedupedRecords, parallelism); + BulkInsertPartitioner>> partitioner = + userDefinedBulkInsertPartitioner.isPresent() + ? userDefinedBulkInsertPartitioner.get() + : convertBulkInsertPartitioner( + BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode())); + repartitionedRecords = getRdd(partitioner.repartitionRecords(dedupedRecords, parallelism)); // generate new file ID prefixes for each output partition final List fileIDPrefixes = IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList()); JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(instantTime, - partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes, useWriterSchema, preserveMetadata), true) + .mapPartitionsWithIndex(new BulkInsertMapFunction<>( + instantTime, partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes, + useWriterSchema, preserveMetadata), true) .flatMap(List::iterator); - return writeStatusRDD; + return SparkHoodieRDDData.of(writeStatusRDD); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java deleted file mode 100644 index e6b6809498e29..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,63 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.BulkInsertPartitioner; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkBulkInsertPreppedCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private final JavaRDD> preppedInputRecordRdd; - private final Option> userDefinedBulkInsertPartitioner; - - public SparkBulkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option> userDefinedBulkInsertPartitioner) { - super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); - this.preppedInputRecordRdd = preppedInputRecordRdd; - this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; - } - - @Override - public HoodieWriteMetadata> execute() { - try { - return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, - this, false, userDefinedBulkInsertPartitioner); - } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } - throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); - } - } - -} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkCommitHelper.java similarity index 61% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkCommitHelper.java index 2bc1f0302798e..70b48a49d4601 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkCommitHelper.java @@ -7,38 +7,35 @@ * "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 + * 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. + * 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.SparkHoodieRDDData; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.client.utils.SparkValidatorUtils; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieUpsertException; @@ -57,57 +54,62 @@ import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.storage.StorageLevel; -import scala.Tuple2; import java.io.IOException; import java.io.Serializable; -import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; -import java.util.stream.Collectors; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; -import java.util.Set; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import scala.Tuple2; -public abstract class BaseSparkCommitActionExecutor extends - BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata> { +import static org.apache.hudi.table.HoodieSparkTable.convertMetadata; - private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); +public class SparkCommitHelper> extends BaseCommitHelper { + + private static final Logger LOG = LogManager.getLogger(SparkCommitHelper.class); protected Option keyGeneratorOpt = Option.empty(); - public BaseSparkCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType) { - super(context, config, table, instantTime, operationType, Option.empty()); - initKeyGenIfNeeded(config.populateMetaFields()); + public SparkCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + this(context, config, table, instantTime, operationType, Option.empty()); } - public BaseSparkCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable table, - String instantTime, - WriteOperationType operationType, - Option extraMetadata) { + public SparkCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, Option> extraMetadata) { super(context, config, table, instantTime, operationType, extraMetadata); + } + + public static JavaRDD getRdd(HoodieData hoodieData) { + return ((SparkHoodieRDDData) hoodieData).get(); + } + + @Override + public void init(HoodieWriteConfig config) { initKeyGenIfNeeded(config.populateMetaFields()); } private void initKeyGenIfNeeded(boolean populateMetaFields) { if (!populateMetaFields) { try { - keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()))); + keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory + .createKeyGenerator(new TypedProperties(config.getProps()))); } catch (IOException e) { throw new HoodieIOException("Only BaseKeyGenerators are supported when meta columns are disabled ", e); } @@ -117,18 +119,20 @@ private void initKeyGenIfNeeded(boolean populateMetaFields) { private JavaRDD> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { if (config.isClusteringEnabled()) { Set fileGroupsInPendingClustering = - table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); - UpdateStrategy updateStrategy = (UpdateStrategy)ReflectionUtils + table.getFileSystemView().getFileGroupsInPendingClustering().map( + entry -> entry.getKey()).collect(Collectors.toSet()); + UpdateStrategy updateStrategy = (UpdateStrategy) ReflectionUtils .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); - return (JavaRDD>)updateStrategy.handleUpdate(inputRecordsRDD); + return (JavaRDD>) updateStrategy.handleUpdate(inputRecordsRDD); } else { return inputRecordsRDD; } } @Override - public HoodieWriteMetadata> execute(JavaRDD> inputRecordsRDD) { - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + public HoodieWriteMetadata> execute(HoodieData> inputRecords) { + JavaRDD> inputRecordsRDD = getRdd(inputRecords); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); // Cache the tagged records, so we don't end up computing both // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling if (inputRecordsRDD.getStorageLevel() == StorageLevel.NONE()) { @@ -205,7 +209,8 @@ protected Partitioner getPartitioner(WorkloadProfile profile) { } } - private JavaRDD> partition(JavaRDD> dedupedRecords, Partitioner partitioner) { + private JavaRDD> partition( + JavaRDD> dedupedRecords, Partitioner partitioner) { JavaPairRDD>, HoodieRecord> mappedRDD = dedupedRecords.mapToPair( record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)); @@ -213,11 +218,12 @@ record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.ge if (table.requireSortedRecords()) { // Partition and sort within each partition as a single step. This is faster than partitioning first and then // applying a sort. - Comparator>> comparator = (Comparator>> & Serializable)(t1, t2) -> { - HoodieKey key1 = t1._1; - HoodieKey key2 = t2._1; - return key1.getRecordKey().compareTo(key2.getRecordKey()); - }; + Comparator>> comparator = + (Comparator>> & Serializable) (t1, t2) -> { + HoodieKey key1 = t1._1; + HoodieKey key2 = t2._1; + return key1.getRecordKey().compareTo(key2.getRecordKey()); + }; partitionedRDD = mappedRDD.repartitionAndSortWithinPartitions(partitioner, comparator); } else { @@ -228,66 +234,35 @@ record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.ge return partitionedRDD.map(Tuple2::_2); } - protected JavaRDD updateIndex(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { + public HoodieData updateIndex( + JavaRDD writeStatusRDD, HoodieWriteMetadata result) { // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future // RDD actions that are performed after updating the index. writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = table.getIndex().updateLocation(writeStatusRDD, context, table); + HoodieData statuses = SparkHoodieRDDData.of((JavaRDD) + table.getIndex().updateLocation(writeStatusRDD, context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); return statuses; } - - protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { + + public void updateIndexAndCommitIfNeeded( + JavaRDD writeStatusRDD, HoodieWriteMetadata> result) { updateIndex(writeStatusRDD, result); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); commitOnAutoCommit(result); } - @Override - protected String getCommitActionType() { - return table.getMetaClient().getCommitActionType(); - } - - @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { - context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect"); - commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect()); - } - - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { - String actionType = getCommitActionType(); - LOG.info("Committing " + instantTime + ", action Type " + actionType); - result.setCommitted(true); - result.setWriteStats(writeStats); - // Finalize write - finalizeWrite(instantTime, writeStats, result); - syncTableMetadata(); - try { - LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), - extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); - - activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - LOG.info("Committed " + instantTime); - result.setCommitMetadata(Option.of(metadata)); - } catch (IOException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); - } - } - - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeStatuses) { + public Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeStatuses) { return Collections.emptyMap(); } @SuppressWarnings("unchecked") - protected Iterator> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr, - Partitioner partitioner) { + protected Iterator> handleUpsertPartition( + String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) { UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); BucketType btype = binfo.bucketType; @@ -306,44 +281,13 @@ protected Iterator> handleUpsertPartition(String instantTime, } } - protected Iterator> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, - Partitioner partitioner) { + protected Iterator> handleInsertPartition( + String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) { return handleUpsertPartition(instantTime, partition, recordItr, partitioner); } - @Override - public Iterator> handleUpdate(String partitionPath, String fileId, - Iterator> recordItr) - throws IOException { - // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records - if (!recordItr.hasNext()) { - LOG.info("Empty partition with fileId => " + fileId); - return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); - } - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); - return handleUpdateInternal(upsertHandle, fileId); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) - throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException( - "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); - } else { - SparkMergeHelper.newInstance().runMerge(table, upsertHandle); - } - - // TODO(vc): This needs to be revisited - if (upsertHandle.getPartitionPath() == null) { - LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.writeStatuses()); - } - - return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); - } - - protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { + protected HoodieMergeHandle getUpdateHandle( + String partitionPath, String fileId, Iterator> recordItr) { if (table.requireSortedRecords()) { return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieSparkTable) table, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt); @@ -355,10 +299,14 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, } @Override + protected BaseMergeHelper getMergeHelper() { + return SparkMergeHelper.newInstance(); + } + public void syncTableMetadata() { // Open up the metadata table again, for syncing try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { + SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context)) { LOG.info("Successfully synced to metadata table"); } catch (Exception e) { throw new HoodieMetadataException("Error syncing to metadata table.", e); @@ -366,8 +314,8 @@ public void syncTableMetadata() { } @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { + public Iterator> handleInsert( + String idPfx, Iterator> recordItr) throws Exception { // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records if (!recordItr.hasNext()) { LOG.info("Empty partition"); @@ -388,8 +336,7 @@ public Partitioner getInsertPartitioner(WorkloadProfile profile) { return getUpsertPartitioner(profile); } - @Override - protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { - SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + SparkValidatorUtils.runValidators(config, convertMetadata(writeMetadata), context, table, instantTime); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java deleted file mode 100644 index 997c7bf2376e3..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java +++ /dev/null @@ -1,48 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkDeleteCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private final JavaRDD keys; - - public SparkDeleteCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { - super(context, config, table, instantTime, WriteOperationType.DELETE); - this.keys = keys; - } - - @Override - public HoodieWriteMetadata> execute() { - return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java index 83ead05085015..17830fe2025d1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -27,6 +28,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -41,13 +43,12 @@ import java.util.HashMap; /** - * A spark implementation of {@link AbstractDeleteHelper}. + * A spark implementation of {@link BaseDeleteHelper}. * * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkDeleteHelper extends - AbstractDeleteHelper>, JavaRDD, JavaRDD, R> { +public class SparkDeleteHelper> extends BaseDeleteHelper { private SparkDeleteHelper() { } @@ -56,60 +57,67 @@ private static class DeleteHelperHolder { } public static SparkDeleteHelper newInstance() { - return DeleteHelperHolder.SPARK_DELETE_HELPER; + return SparkDeleteHelper.DeleteHelperHolder.SPARK_DELETE_HELPER; } @Override - public JavaRDD deduplicateKeys(JavaRDD keys, HoodieTable>, JavaRDD, JavaRDD> table, int parallelism) { + public SparkHoodieRDDData deduplicateKeys( + HoodieData keys, + HoodieTable>, HoodieData, HoodieData> table, + int parallelism) { + JavaRDD keysRdd = ((SparkHoodieRDDData) keys).get(); boolean isIndexingGlobal = table.getIndex().isGlobal(); if (isIndexingGlobal) { - return keys.keyBy(HoodieKey::getRecordKey) + return SparkHoodieRDDData.of(keysRdd.keyBy(HoodieKey::getRecordKey) .reduceByKey((key1, key2) -> key1, parallelism) - .values(); + .values()); } else { - return keys.distinct(parallelism); + return SparkHoodieRDDData.of(keysRdd.distinct(parallelism)); } } @Override - public HoodieWriteMetadata> execute(String instantTime, - JavaRDD keys, - HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable>, JavaRDD, JavaRDD> table, - BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> deleteExecutor) { + public HoodieWriteMetadata> execute( + String instantTime, HoodieData keys, HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + BaseCommitHelper commitHelper) { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); try { - HoodieWriteMetadata result = null; - JavaRDD dedupedKeys = keys; + HoodieWriteMetadata> result = null; + SparkHoodieRDDData dedupedKeys = (SparkHoodieRDDData) keys; final int parallelism = config.getDeleteShuffleParallelism(); if (config.shouldCombineBeforeDelete()) { // De-dupe/merge if needed dedupedKeys = deduplicateKeys(keys, table, parallelism); - } else if (!keys.partitions().isEmpty()) { - dedupedKeys = keys.repartition(parallelism); + } else { + JavaRDD keysRdd = ((SparkHoodieRDDData) keys).get(); + if (!keysRdd.partitions().isEmpty()) { + dedupedKeys = SparkHoodieRDDData.of(keysRdd.repartition(parallelism)); + } } JavaRDD> dedupedRecords = - dedupedKeys.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); + dedupedKeys.get().map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); Instant beginTag = Instant.now(); // perform index loop up to get existing location of records - JavaRDD> taggedRecords = - table.getIndex().tagLocation(dedupedRecords, context, table); + HoodieData> taggedRecords = SparkHoodieRDDData.of((JavaRDD>) + table.getIndex().tagLocation(dedupedRecords, context, table)); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records - JavaRDD> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown); + HoodieData> taggedValidRecords = SparkHoodieRDDData.of( + ((SparkHoodieRDDData>) taggedRecords).get().filter(HoodieRecord::isCurrentLocationKnown)); if (!taggedValidRecords.isEmpty()) { - result = deleteExecutor.execute(taggedValidRecords); + result = commitHelper.execute(taggedValidRecords); result.setIndexLookupDuration(tagLocationDuration); } else { // if entire set of keys are non existent - deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); - result = new HoodieWriteMetadata(); - result.setWriteStatuses(jsc.emptyRDD()); - deleteExecutor.commitOnAutoCommit(result); + commitHelper.saveWorkloadProfileMetadataToInflight(new WorkloadProfile( + Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + result = new HoodieWriteMetadata<>(); + result.setWriteStatuses(SparkHoodieRDDData.of(jsc.emptyRDD())); + commitHelper.commitOnAutoCommit(result); } return result; } catch (Throwable e) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java deleted file mode 100644 index 90bcdc9b9c141..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java +++ /dev/null @@ -1,68 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.HoodieTimer; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.WorkloadProfile; -import org.apache.hudi.table.WorkloadStat; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import scala.Tuple2; - -import java.time.Duration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class SparkDeletePartitionCommitActionExecutor> - extends SparkInsertOverwriteCommitActionExecutor { - - private List partitions; - public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, List partitions) { - super(context, config, table, instantTime,null, WriteOperationType.DELETE_PARTITION); - this.partitions = partitions; - } - - @Override - public HoodieWriteMetadata> execute() { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); - HoodieTimer timer = new HoodieTimer().startTimer(); - Map> partitionToReplaceFileIds = jsc.parallelize(partitions, partitions.size()).distinct() - .mapToPair(partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); - HoodieWriteMetadata result = new HoodieWriteMetadata(); - result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); - result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); - - result.setWriteStatuses(jsc.emptyRDD()); - this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); - this.commitOnAutoCommit(result); - return result; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionHelper.java new file mode 100644 index 0000000000000..eb0d64a58176f --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionHelper.java @@ -0,0 +1,82 @@ +/* + * 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.SparkHoodieRDDData; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.spark.api.java.JavaSparkContext; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import scala.Tuple2; + +public class SparkDeletePartitionHelper> + extends BaseDeletePartitionHelper { + + private SparkDeletePartitionHelper() { + } + + private static class DeletePartitionHelperHolder { + private static final SparkDeletePartitionHelper SPARK_DELETE_PARTITION_HELPER = new SparkDeletePartitionHelper(); + } + + public static SparkDeletePartitionHelper newInstance() { + return SparkDeletePartitionHelper.DeletePartitionHelperHolder.SPARK_DELETE_PARTITION_HELPER; + } + + public HoodieWriteMetadata> execute( + String instantTime, HoodieEngineContext context, HoodieTable table, List partitions, + BaseCommitHelper commitHelper) { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + HoodieTimer timer = new HoodieTimer().startTimer(); + Map> partitionToReplaceFileIds = jsc.parallelize(partitions, partitions.size()).distinct() + .mapToPair(partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(table, partitionPath))).collectAsMap(); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); + + result.setWriteStatuses(SparkHoodieRDDData.of(jsc.emptyRDD())); + commitHelper.saveWorkloadProfileMetadataToInflight( + new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + commitHelper.commitOnAutoCommit(result); + return result; + } + + protected List getAllExistingFileIds(HoodieTable table, String partitionPath) { + // because new commit is not complete. it is safe to mark all existing file Ids as old files + return table.getSliceView().getLatestFileSlices(partitionPath).map( + fg -> fg.getFileId()).distinct().collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java deleted file mode 100644 index 25891e05a4dd1..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkInsertCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private final JavaRDD> inputRecordsRDD; - - public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(context, config, table, instantTime, WriteOperationType.INSERT); - this.inputRecordsRDD = inputRecordsRDD; - } - - @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java deleted file mode 100644 index bff85e7fe1c91..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ /dev/null @@ -1,84 +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.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.WorkloadProfile; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaRDD; -import scala.Tuple2; - -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class SparkInsertOverwriteCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private final JavaRDD> inputRecordsRDD; - - public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - this(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE); - } - - public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - WriteOperationType writeOperationType) { - super(context, config, table, instantTime, writeOperationType); - this.inputRecordsRDD = inputRecordsRDD; - } - - @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); - } - - @Override - protected Partitioner getPartitioner(WorkloadProfile profile) { - return new SparkInsertOverwritePartitioner(profile, context, table, config); - } - - @Override - protected String getCommitActionType() { - return HoodieTimeline.REPLACE_COMMIT_ACTION; - } - - @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - return writeMetadata.getWriteStatuses().map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> - new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); - } - - protected List getAllExistingFileIds(String partitionPath) { - // because new commit is not complete. it is safe to mark all existing file Ids as old files - return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitHelper.java new file mode 100644 index 0000000000000..f0281b9b035ae --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitHelper.java @@ -0,0 +1,71 @@ +/* + * 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.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.spark.Partitioner; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import scala.Tuple2; + +public class SparkInsertOverwriteCommitHelper> extends SparkCommitHelper { + public SparkInsertOverwriteCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + super(context, config, table, instantTime, operationType); + } + + @Override + protected Partitioner getPartitioner(WorkloadProfile profile) { + return new SparkInsertOverwritePartitioner(profile, context, table, config); + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + @Override + public Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeMetadata) { + return getRdd(writeMetadata.getWriteStatuses()).map( + status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> + new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + } + + protected List getAllExistingFileIds(String partitionPath) { + // because new commit is not complete. it is safe to mark all existing file Ids as old files + return table.getSliceView().getLatestFileSlices(partitionPath).map( + fg -> fg.getFileId()).distinct().collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitHelper.java similarity index 59% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitHelper.java index f7c98d5373360..7b6fef671134f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitHelper.java @@ -7,13 +7,14 @@ * "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 + * 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. + * 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; @@ -22,33 +23,36 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import scala.Tuple2; import java.util.HashMap; import java.util.List; import java.util.Map; -public class SparkInsertOverwriteTableCommitActionExecutor> - extends SparkInsertOverwriteCommitActionExecutor { +import scala.Tuple2; - public SparkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE_TABLE); +public class SparkInsertOverwriteTableCommitHelper> + extends SparkInsertOverwriteCommitHelper { + public SparkInsertOverwriteTableCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, + WriteOperationType operationType) { + super(context, config, table, instantTime, operationType); } @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + public Map> getPartitionToReplacedFileIds( + HoodieWriteMetadata> writeMetadata) { Map> partitionToExistingFileIds = new HashMap<>(); - List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); + List partitionPaths = FSUtils.getAllPartitionPaths( + context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); if (partitionPaths != null && partitionPaths.size() > 0) { context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java deleted file mode 100644 index 400147bb8fe70..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,48 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkInsertPreppedCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private final JavaRDD> preppedRecords; - - public SparkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java index 5e82dbd8c566d..80615451374b6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java @@ -43,7 +43,7 @@ import java.io.IOException; import java.util.Iterator; -public class SparkMergeHelper extends AbstractMergeHelper>, +public class SparkMergeHelper extends BaseMergeHelper>, JavaRDD, JavaRDD> { private SparkMergeHelper() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java deleted file mode 100644 index fe90212b0be15..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkUpsertCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private JavaRDD> inputRecordsRDD; - - public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(context, config, table, instantTime, WriteOperationType.UPSERT); - this.inputRecordsRDD = inputRecordsRDD; - } - - @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java deleted file mode 100644 index e36073fd17d6d..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java +++ /dev/null @@ -1,48 +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.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkUpsertPreppedCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - - private final JavaRDD> preppedRecords; - - public SparkUpsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java index 38820be534fa3..9b810ab2d917a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java @@ -18,23 +18,27 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.SparkHoodieRDDData; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.JavaRDD; import scala.Tuple2; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; + /** - * A spark implementation of {@link AbstractWriteHelper}. + * A spark implementation of {@link BaseWriteHelper}. * * @param */ -public class SparkWriteHelper extends AbstractWriteHelper>, - JavaRDD, JavaRDD, R> { +public class SparkWriteHelper> extends BaseWriteHelper { private SparkWriteHelper() { } @@ -47,11 +51,20 @@ public static SparkWriteHelper newInstance() { } @Override - public JavaRDD> deduplicateRecords(JavaRDD> records, - HoodieIndex>, JavaRDD, JavaRDD> index, - int parallelism) { + protected HoodieData> tag( + HoodieData> dedupedRecords, HoodieEngineContext context, HoodieTable table) { + // perform index loop up to get existing location of records + return SparkHoodieRDDData.of( + (JavaRDD>) table.getIndex().tagLocation(getRdd(dedupedRecords), context, table)); + } + + @Override + public HoodieData> deduplicateRecords(HoodieData> records, + HoodieIndex index, + int parallelism) { boolean isIndexingGlobal = index.isGlobal(); - return records.mapToPair(record -> { + JavaRDD> recordsRdd = getRdd(records); + return SparkHoodieRDDData.of(recordsRdd.mapToPair(record -> { HoodieKey hoodieKey = record.getKey(); // If index used is global, then records are expected to differ in their partitionPath Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; @@ -62,7 +75,7 @@ public JavaRDD> deduplicateRecords(JavaRDD> reco HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); return new HoodieRecord(reducedKey, reducedData); - }, parallelism).map(Tuple2::_2); + }, parallelism).map(Tuple2::_2)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java deleted file mode 100644 index 281304d957620..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java +++ /dev/null @@ -1,70 +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.deltacommit; - -import java.util.Map; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.BulkInsertPartitioner; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.spark.api.java.JavaRDD; - -public class SparkBulkInsertDeltaCommitActionExecutor> - extends AbstractSparkDeltaCommitActionExecutor { - - private final JavaRDD> inputRecordsRDD; - private final Option> bulkInsertPartitioner; - - public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner) { - this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); - } - - public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option> bulkInsertPartitioner, - Option> extraMetadata) { - super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); - this.inputRecordsRDD = inputRecordsRDD; - this.bulkInsertPartitioner = bulkInsertPartitioner; - } - - @Override - public HoodieWriteMetadata> execute() { - try { - return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, - this, true, bulkInsertPartitioner); - } catch (HoodieInsertException ie) { - throw ie; - } catch (Throwable e) { - throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java deleted file mode 100644 index 21fc013af69c9..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java +++ /dev/null @@ -1,64 +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.deltacommit; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.BulkInsertPartitioner; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.spark.api.java.JavaRDD; - -public class SparkBulkInsertPreppedDeltaCommitActionExecutor> - extends AbstractSparkDeltaCommitActionExecutor { - - private final JavaRDD> preppedInputRecordRdd; - private final Option> bulkInsertPartitioner; - - public SparkBulkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option> bulkInsertPartitioner) { - super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); - this.preppedInputRecordRdd = preppedInputRecordRdd; - this.bulkInsertPartitioner = bulkInsertPartitioner; - } - - @Override - public HoodieWriteMetadata> execute() { - try { - return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, - this, false, bulkInsertPartitioner); - } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } - throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); - } - } - -} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java deleted file mode 100644 index 4fb6a90f90a41..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.deltacommit; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkDeleteHelper; -import org.apache.spark.api.java.JavaRDD; - -public class SparkDeleteDeltaCommitActionExecutor> - extends AbstractSparkDeltaCommitActionExecutor { - - private final JavaRDD keys; - - public SparkDeleteDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { - super(context, config, table, instantTime, WriteOperationType.DELETE); - this.keys = keys; - } - - @Override - public HoodieWriteMetadata> execute() { - return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeltaCommitHelper.java similarity index 67% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeltaCommitHelper.java index c92cd928474cc..3455d544574cf 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeltaCommitHelper.java @@ -7,20 +7,21 @@ * "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 + * 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. + * 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.deltacommit; -import java.util.Map; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -32,7 +33,7 @@ import org.apache.hudi.io.HoodieAppendHandle; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; -import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkCommitHelper; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -42,22 +43,25 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; -public abstract class AbstractSparkDeltaCommitActionExecutor> - extends BaseSparkCommitActionExecutor { - private static final Logger LOG = LogManager.getLogger(AbstractSparkDeltaCommitActionExecutor.class); +public class SparkDeltaCommitHelper> extends SparkCommitHelper { + + private static final Logger LOG = LogManager.getLogger(SparkDeltaCommitHelper.class); // UpsertPartitioner for MergeOnRead table type private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; - public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType) { + public SparkDeltaCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { this(context, config, table, instantTime, operationType, Option.empty()); } - public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType, - Option> extraMetadata) { + public SparkDeltaCommitHelper( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { super(context, config, table, instantTime, operationType, extraMetadata); } @@ -66,12 +70,14 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { if (profile == null) { throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); } - mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner(profile, (HoodieSparkEngineContext) context, table, config); + mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner( + profile, (HoodieSparkEngineContext) context, table, config); return mergeOnReadUpsertPartitioner; } @Override - public Iterator> handleUpdate(String partitionPath, String fileId, + public Iterator> handleUpdate( + String partitionPath, String fileId, Iterator> recordItr) throws IOException { LOG.info("Merging updates for commit " + instantTime + " for file " + fileId); @@ -79,7 +85,7 @@ public Iterator> handleUpdate(String partitionPath, String fil LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId); return super.handleUpdate(partitionPath, fileId, recordItr); } else { - HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table, + HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table, partitionPath, fileId, recordItr, taskContextSupplier); appendHandle.doAppend(); return Collections.singletonList(appendHandle.close()).iterator(); @@ -97,5 +103,4 @@ public Iterator> handleInsert(String idPfx, Iterator> - extends AbstractSparkDeltaCommitActionExecutor { - - private final JavaRDD> inputRecordsRDD; - - public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(context, config, table, instantTime, WriteOperationType.INSERT); - this.inputRecordsRDD = inputRecordsRDD; - } - - @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, false); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java deleted file mode 100644 index 1f1e0165b494a..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java +++ /dev/null @@ -1,47 +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.deltacommit; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkInsertPreppedDeltaCommitActionExecutor> - extends AbstractSparkDeltaCommitActionExecutor { - - private final JavaRDD> preppedRecords; - - public SparkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java deleted file mode 100644 index 82aa081524050..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java +++ /dev/null @@ -1,49 +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.deltacommit; - -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; - -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkWriteHelper; -import org.apache.spark.api.java.JavaRDD; - -public class SparkUpsertDeltaCommitActionExecutor> - extends AbstractSparkDeltaCommitActionExecutor { - - private JavaRDD> inputRecordsRDD; - - public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { - super(context, config, table, instantTime, WriteOperationType.UPSERT); - this.inputRecordsRDD = inputRecordsRDD; - } - - @Override - public HoodieWriteMetadata execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, true); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java deleted file mode 100644 index 3509efa6bfa9f..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java +++ /dev/null @@ -1,47 +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.deltacommit; - -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - -public class SparkUpsertPreppedDeltaCommitActionExecutor> - extends AbstractSparkDeltaCommitActionExecutor { - - private final JavaRDD> preppedRecords; - - public SparkUpsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { - super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); - this.preppedRecords = preppedRecords; - } - - @Override - public HoodieWriteMetadata> execute() { - return super.execute(preppedRecords); - } -} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index de4f42177429a..e58e0991e6090 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -18,6 +18,7 @@ package org.apache.hudi.client.functional; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; @@ -138,6 +139,7 @@ import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet; import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; import static org.apache.hudi.config.HoodieClusteringConfig.EXECUTION_STRATEGY_CLASS_NAME; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -441,7 +443,9 @@ private void testDeduplication( // Global dedup should be done based on recordKey only HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(true); - List> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); + List> dedupedRecs = getRdd( + SparkWriteHelper.newInstance().deduplicateRecords( + SparkHoodieRDDData.of(records), index, 1)).collect(); assertEquals(1, dedupedRecs.size()); assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath()); assertNodupesWithinPartition(dedupedRecs); @@ -449,7 +453,8 @@ private void testDeduplication( // non-Global dedup should be done based on both recordKey and partitionPath index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(false); - dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); + dedupedRecs = getRdd(SparkWriteHelper.newInstance().deduplicateRecords( + SparkHoodieRDDData.of(records), index, 1)).collect(); assertEquals(2, dedupedRecs.size()); assertNodupesWithinPartition(dedupedRecs); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index f0046afe03a70..5c308a7790fcd 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -40,8 +41,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.table.action.deltacommit.AbstractSparkDeltaCommitActionExecutor; -import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.SparkDeltaCommitHelper; import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; @@ -524,11 +524,11 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception { JavaRDD deleteRDD = jsc().parallelize(fewRecordsForDelete, 1); // initialize partitioner - AbstractSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable, - newDeleteTime, deleteRDD); - actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD))); + SparkDeltaCommitHelper deltaCommitHelper = new SparkDeltaCommitHelper( + context(), cfg, hoodieTable, newDeleteTime, WriteOperationType.DELETE); + deltaCommitHelper.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD))); final List> deleteStatus = jsc().parallelize(Arrays.asList(1)).map(x -> { - return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator()); + return deltaCommitHelper.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator()); }).map(Transformations::flatten).collect(); // Verify there are errors because records are from multiple partitions (but handleUpdate is invoked for @@ -536,7 +536,7 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception { WriteStatus status = deleteStatus.get(0).get(0); assertTrue(status.hasErrors()); long numRecordsInPartition = fewRecordsForDelete.stream().filter(u -> - u.getPartitionPath().equals(partitionPath)).count(); + u.getPartitionPath().equals(partitionPath)).count(); assertEquals(fewRecordsForDelete.size() - numRecordsInPartition, status.getTotalErrorRecords()); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 40df1af898ea3..0654b9efeaf36 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; @@ -25,6 +26,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; @@ -34,6 +36,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieData; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.io.HoodieCreateHandle; @@ -73,6 +76,7 @@ import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateExpectedPartitionNumRecords; import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateTestRecordsForBulkInsert; +import static org.apache.hudi.table.action.commit.SparkCommitHelper.getRdd; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -287,8 +291,8 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records - BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, - firstCommitTime, jsc.parallelize(records)); + SparkCommitHelper actionExecutor = new SparkCommitHelper<>(context, config, table, + firstCommitTime, WriteOperationType.INSERT); List writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -330,8 +334,8 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs2 = records; - BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(recs2)); + SparkCommitHelper actionExecutor = new SparkCommitHelper(context, config, table, + instantTime, WriteOperationType.INSERT_PREPPED); List returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -351,10 +355,10 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs3 = records; - BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(recs3)); + SparkCommitHelper sparkCommitHelper = new SparkCommitHelper(context, config, table, + instantTime, WriteOperationType.UPSERT_PREPPED); returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { - return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()); + return sparkCommitHelper.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); assertEquals(3, returnedStatuses.size()); @@ -384,10 +388,10 @@ public void testFileSizeUpsertRecords() throws Exception { } // Insert new records - BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(records)); + SparkCommitHelper sparkCommitHelper = new SparkCommitHelper(context, config, table, + instantTime, WriteOperationType.UPSERT); jsc.parallelize(Arrays.asList(1)) - .map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) + .map(i -> sparkCommitHelper.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) .map(Transformations::flatten).collect(); // Check the updated file @@ -405,15 +409,15 @@ public void testFileSizeUpsertRecords() throws Exception { public void testInsertUpsertWithHoodieAvroPayload() throws Exception { Schema schema = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/testDataGeneratorSchema.txt"); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()) - .withStorageConfig(HoodieStorageConfig.newBuilder() - .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build(); + .withStorageConfig(HoodieStorageConfig.newBuilder() + .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); String instantTime = "000"; // Perform inserts of 100 records to test CreateHandle and BufferedExecutor final List inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100); - BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(inserts)); + SparkCommitHelper actionExecutor = new SparkCommitHelper<>(context, config, table, + instantTime, WriteOperationType.INSERT); final List> ws = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator()); }).map(Transformations::flatten).collect(); @@ -426,10 +430,10 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { String partitionPath = writeStatus.getPartitionPath(); long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, HoodieTableMetaClient.reload(metaClient)); - BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(updates)); + SparkCommitHelper sparkCommitHelper = new SparkCommitHelper(context, config, table, + instantTime, WriteOperationType.UPSERT); final List> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> { - return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator()); + return sparkCommitHelper.handleUpdate(partitionPath, fileId, updates.iterator()); }).map(Transformations::flatten).collect(); assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords()); } @@ -446,9 +450,11 @@ public void testBulkInsertRecords(String bulkInsertMode) throws Exception { // Insert new records final JavaRDD inputRecords = generateTestRecordsForBulkInsert(jsc); - SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor( - context, config, table, instantTime, inputRecords, Option.empty()); - List returnedStatuses = ((JavaRDD)bulkInsertExecutor.execute().getWriteStatuses()).collect(); + BulkInsertCommitActionExecutor bulkInsertExecutor = new BulkInsertCommitActionExecutor( + context, config, table, instantTime, SparkHoodieRDDData.of(inputRecords), Option.empty(), + new SparkCommitHelper(context, config, table, instantTime, WriteOperationType.BULK_INSERT), + SparkBulkInsertHelper.newInstance()); + List returnedStatuses = (getRdd((HoodieData) bulkInsertExecutor.execute().getWriteStatuses())).collect(); verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords)); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java index d9dc6ac978d92..c613d34dec374 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java @@ -17,6 +17,7 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.SparkHoodieRDDData; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; @@ -64,13 +65,21 @@ private enum CombineTestMode { private static final boolean WITHOUT_COMBINE = false; private static final int DELETE_PARALLELISM = 200; - @Mock private SparkHoodieBloomIndex index; - @Mock private HoodieTable, JavaRDD, JavaRDD> table; - @Mock private BaseSparkCommitActionExecutor executor; - @Mock private HoodieWriteMetadata metadata; - @Mock private JavaPairRDD keyPairs; - @Mock private JavaSparkContext jsc; - @Mock private HoodieSparkEngineContext context; + @Mock + private SparkHoodieBloomIndex index; + @Mock + private HoodieTable, JavaRDD, JavaRDD> table; + //@Mock private BaseSparkCommitActionExecutor executor; + @Mock + private SparkCommitHelper commitHelper; + @Mock + private HoodieWriteMetadata metadata; + @Mock + private JavaPairRDD keyPairs; + @Mock + private JavaSparkContext jsc; + @Mock + private HoodieSparkEngineContext context; private JavaRDD rddToDelete; private HoodieWriteConfig config; @@ -86,7 +95,8 @@ public void deleteWithEmptyRDDShouldNotExecute() { rddToDelete = mockEmptyHoodieKeyRdd(); config = newWriteConfig(WITHOUT_COMBINE); - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); + SparkDeleteHelper.newInstance().execute( + "test-time", SparkHoodieRDDData.of(rddToDelete), context, config, table, commitHelper); verify(rddToDelete, never()).repartition(DELETE_PARALLELISM); verifyNoDeleteExecution(); @@ -97,7 +107,8 @@ public void deleteWithoutCombineShouldRepartitionForNonEmptyRdd() { rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None); config = newWriteConfig(WITHOUT_COMBINE); - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); + SparkDeleteHelper.newInstance().execute( + "test-time", SparkHoodieRDDData.of(rddToDelete), context, config, table, commitHelper); verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM); verifyDeleteExecution(); @@ -108,7 +119,8 @@ public void deleteWithCombineShouldRepartitionForNonEmptyRddAndNonGlobalIndex() rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex); config = newWriteConfig(WITH_COMBINE); - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); + SparkDeleteHelper.newInstance().execute( + "test-time", SparkHoodieRDDData.of(rddToDelete), context, config, table, commitHelper); verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM); verifyDeleteExecution(); @@ -120,19 +132,20 @@ public void deleteWithCombineShouldRepartitionForNonEmptyRddAndGlobalIndex() { config = newWriteConfig(WITH_COMBINE); when(index.isGlobal()).thenReturn(true); - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); + SparkDeleteHelper.newInstance().execute( + "test-time", SparkHoodieRDDData.of(rddToDelete), context, config, table, commitHelper); verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM)); verifyDeleteExecution(); } private void verifyDeleteExecution() { - verify(executor, times(1)).execute(any()); + verify(commitHelper, times(1)).execute(any()); verify(metadata, times(1)).setIndexLookupDuration(any()); } private void verifyNoDeleteExecution() { - verify(executor, never()).execute(any()); + verify(commitHelper, never()).execute(any()); } private HoodieWriteConfig newWriteConfig(boolean combine) { @@ -165,7 +178,7 @@ private JavaRDD newHoodieKeysRddMock(int howMany, CombineTestMode com } when(keysToDelete.map(any())).thenReturn(recordsRdd); - when(executor.execute(any())).thenReturn(metadata); + when(commitHelper.execute(any())).thenReturn(metadata); return keysToDelete; } @@ -178,7 +191,7 @@ private JavaRDD mockEmptyHoodieKeyRdd() { doReturn(emptyRdd).when(index).tagLocation(any(), any(), any()); doReturn(emptyRdd).when(emptyRdd).filter(any()); - doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString()); + doNothing().when(commitHelper).saveWorkloadProfileMetadataToInflight(any(), anyString()); doReturn(emptyRdd).when(jsc).emptyRDD(); return emptyRdd; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java index 10c7ced070387..5f36e6f432fcb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; import java.util.List; import java.util.Map; @@ -56,6 +57,8 @@ public TaskContextSupplier getTaskContextSupplier() { return taskContextSupplier; } + public abstract List map(HoodieData data, SerializableFunction func); + public abstract List map(List data, SerializableFunction func, int parallelism); public abstract List mapToPairAndReduceByKey( diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java index 1c935ff06e4b2..c19bf3e6fcaf6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.engine; import org.apache.hadoop.conf.Configuration; + import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableConsumer; @@ -27,6 +28,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.data.HoodieData; +import org.apache.hudi.data.HoodieListData; import java.util.List; import java.util.Map; @@ -54,6 +57,15 @@ public HoodieLocalEngineContext(Configuration conf, TaskContextSupplier taskCont super(new SerializableConfiguration(conf), taskContextSupplier); } + public static List getList(HoodieData hoodieData) { + return ((HoodieListData) hoodieData).get(); + } + + @Override + public List map(HoodieData data, SerializableFunction func) { + return getList(data).stream().map(throwingMapWrapper(func)).collect(Collectors.toList()); + } + @Override public List map(List data, SerializableFunction func, int parallelism) { return data.stream().parallel().map(throwingMapWrapper(func)).collect(toList()); diff --git a/hudi-common/src/main/java/org/apache/hudi/data/HoodieData.java b/hudi-common/src/main/java/org/apache/hudi/data/HoodieData.java new file mode 100644 index 0000000000000..73584d0e69652 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/data/HoodieData.java @@ -0,0 +1,26 @@ +/* + * 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.data; + +public abstract class HoodieData { + public abstract Object get(); + + public abstract boolean isEmpty(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/data/HoodieListData.java b/hudi-common/src/main/java/org/apache/hudi/data/HoodieListData.java new file mode 100644 index 0000000000000..f982e552a8830 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/data/HoodieListData.java @@ -0,0 +1,45 @@ +/* + * 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.data; + +import java.util.List; + +public class HoodieListData extends HoodieData { + + private final List listData; + + private HoodieListData(List listData) { + this.listData = listData; + } + + public static HoodieListData of(List listData) { + return new HoodieListData<>(listData); + } + + @Override + public List get() { + return listData; + } + + @Override + public boolean isEmpty() { + return listData.isEmpty(); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java index c71a91a82d106..067b8de93f733 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java @@ -28,10 +28,12 @@ import org.apache.hudi.common.util.ObjectSizeCalculator; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.data.HoodieListData; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; import org.apache.hudi.sink.event.WriteMetadataEvent; +import org.apache.hudi.table.action.commit.FlinkCommitHelper; import org.apache.hudi.table.action.commit.FlinkWriteHelper; import org.apache.hudi.util.StreamerUtil; @@ -419,7 +421,8 @@ private boolean flushBucket(DataBucket bucket) { List records = bucket.writeBuffer(); ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records"); if (config.getBoolean(FlinkOptions.INSERT_DROP_DUPS)) { - records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1); + records = FlinkCommitHelper.getList( + FlinkWriteHelper.newInstance().deduplicateRecords(HoodieListData.of(records), (HoodieIndex) null, -1)); } bucket.preWrite(records); final List writeStatus = new ArrayList<>(writeFunction.apply(records, instant)); @@ -454,7 +457,8 @@ private void flushRemaining(boolean endInput) { List records = bucket.writeBuffer(); if (records.size() > 0) { if (config.getBoolean(FlinkOptions.INSERT_DROP_DUPS)) { - records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1); + records = FlinkCommitHelper.getList( + FlinkWriteHelper.newInstance().deduplicateRecords(HoodieListData.of(records), (HoodieIndex) null, -1)); } bucket.preWrite(records); writeStatus.addAll(writeFunction.apply(records, currentInstant));