From 461047d03e9b7b73b34536ccf290a24f4aada6b0 Mon Sep 17 00:00:00 2001 From: Nishith Date: Wed, 10 Mar 2021 23:18:23 -0800 Subject: [PATCH 1/2] [HUDI-845] Added locking capability to allow multiple writers 1. Added LockProvider API for pluggable lock methodologies 2. Added Resolution Strategy API to allow for pluggable conflict resolution 3. Added TableService client API to schedule table services 4. Added Transaction Manager for wrapping actions within transactions --- .../testutils/HoodieTestCommitUtilities.java | 4 +- hudi-client/hudi-client-common/pom.xml | 24 + .../client/AbstractHoodieWriteClient.java | 211 +++++++-- .../transaction/ConcurrentOperation.java | 153 +++++++ .../ConflictResolutionStrategy.java | 64 +++ ...tFileWritesConflictResolutionStrategy.java | 107 +++++ .../transaction/TransactionManager.java | 94 ++++ .../client/transaction/lock/LockManager.java | 135 ++++++ .../lock/ZookeeperBasedLockProvider.java | 168 +++++++ .../client/utils/MetadataConversionUtils.java | 132 ++++++ .../hudi/client/utils/TransactionUtils.java | 71 +++ .../hudi/config/HoodieCompactionConfig.java | 1 - .../apache/hudi/config/HoodieLockConfig.java | 191 ++++++++ .../apache/hudi/config/HoodieWriteConfig.java | 82 +++- .../HoodieBackedTableMetadataWriter.java | 7 +- .../org/apache/hudi/table/HoodieTable.java | 14 + .../hudi/table/HoodieTimelineArchiveLog.java | 82 +--- .../action/clean/BaseCleanActionExecutor.java | 93 +--- .../clean/BaseCleanPlanActionExecutor.java | 132 ++++++ .../commit/BaseCommitActionExecutor.java | 27 +- .../BaseScheduleCompactionActionExecutor.java | 34 +- .../upgrade/AbstractUpgradeDowngrade.java | 8 +- .../FileSystemBasedLockProviderTestClass.java | 110 +++++ ...tFileWritesConflictResolutionStrategy.java | 420 ++++++++++++++++++ .../TestZookeeperBasedLockProvider.java | 116 +++++ .../utils/TestMetadataConversionUtils.java | 213 +++++++++ .../hudi/client/HoodieFlinkWriteClient.java | 16 +- .../table/HoodieFlinkCopyOnWriteTable.java | 13 + .../FlinkScheduleCleanActionExecutor.java | 52 +++ .../commit/BaseFlinkCommitActionExecutor.java | 2 +- .../hudi/client/HoodieJavaWriteClient.java | 10 +- .../table/HoodieJavaCopyOnWriteTable.java | 7 + .../JavaScheduleCleanActionExecutor.java | 52 +++ .../commit/BaseJavaCommitActionExecutor.java | 1 - .../TestJavaCopyOnWriteActionExecutor.java | 5 +- .../hudi/client/SparkRDDWriteClient.java | 81 +++- .../table/HoodieSparkCopyOnWriteTable.java | 9 +- .../SparkBootstrapCommitActionExecutor.java | 16 +- .../clean/SparkCleanPlanActionExecutor.java | 55 +++ .../commit/BaseSparkCommitActionExecutor.java | 16 +- .../client/TestHoodieClientMultiWriter.java | 299 +++++++++++++ .../TestHoodieClientOnCopyOnWriteStorage.java | 6 +- .../hudi/io/TestHoodieTimelineArchiveLog.java | 4 +- .../metadata/TestHoodieBackedMetadata.java | 4 - .../org/apache/hudi/table/TestCleaner.java | 9 +- .../commit/TestCopyOnWriteActionExecutor.java | 3 +- .../testutils/HoodieClientTestHarness.java | 15 +- .../hudi/common/config/LockConfiguration.java | 69 +++ .../apache/hudi/common/lock/LockProvider.java | 56 +++ .../apache/hudi/common/lock/LockState.java | 27 ++ .../common/model/HoodieMetadataWrapper.java | 49 ++ .../hudi/common/model/TableServiceType.java | 41 ++ .../common/model/WriteConcurrencyMode.java | 66 +++ .../hudi/common/model/WriteOperationType.java | 6 + .../table/timeline/HoodieDefaultTimeline.java | 6 + .../common/table/timeline/HoodieTimeline.java | 5 + .../hudi/common/util/ClusteringUtils.java | 53 ++- .../apache/hudi/common/util/CommitUtils.java | 25 ++ .../hudi/exception/HoodieLockException.java | 43 ++ .../HoodieWriteConflictException.java | 43 ++ .../common/testutils/FileCreateUtils.java | 22 +- .../common/testutils/HoodieTestTable.java | 28 ++ .../minicluster/ZookeeperTestService.java | 4 + .../hudi/common/util/TestCommitUtils.java | 2 +- hudi-integ-test/pom.xml | 44 +- .../integ/testsuite/HoodieTestSuiteJob.java | 35 +- .../testsuite/HoodieTestSuiteWriter.java | 24 +- .../integ/testsuite/dag/WriterContext.java | 4 + .../helpers/ZookeeperServiceProvider.java | 50 +++ .../writer/DFSDeltaWriterAdapter.java | 1 - .../testsuite/dag/nodes/SparkInsertNode.scala | 65 +++ .../testsuite/dag/nodes/SparkUpsertNode.scala | 65 +++ .../testsuite/job/TestHoodieTestSuiteJob.java | 103 +++-- .../unit-test-cow-dag-spark-datasource.yaml | 43 ++ hudi-spark-datasource/hudi-spark/pom.xml | 24 + .../apache/hudi/HoodieSparkSqlWriter.scala | 4 +- .../hudi/hive/HiveMetastoreLockProvider.java | 226 ++++++++++ .../hive/TestHiveMetastoreLockProvider.java | 158 +++++++ .../hudi/hive/testutils/HiveTestService.java | 6 +- .../hudi/hive/testutils/HiveTestUtil.java | 15 +- .../utilities/deltastreamer/DeltaSync.java | 2 +- packaging/hudi-integ-test-bundle/pom.xml | 22 + packaging/hudi-spark-bundle/pom.xml | 24 +- pom.xml | 20 + 84 files changed, 4487 insertions(+), 361 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanPlanActionExecutor.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java create mode 100644 hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkScheduleCleanActionExecutor.java create mode 100644 hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaScheduleCleanActionExecutor.java create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanPlanActionExecutor.java create mode 100644 hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/lock/LockState.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMetadataWrapper.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/model/WriteConcurrencyMode.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/exception/HoodieLockException.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java create mode 100644 hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.java create mode 100644 hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala create mode 100644 hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala create mode 100644 hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml create mode 100644 hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java create mode 100644 hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitUtilities.java b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitUtilities.java index 8829c858051fb..13f8a74faa5d9 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitUtilities.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitUtilities.java @@ -19,8 +19,8 @@ package org.apache.hudi.cli.testutils; import org.apache.hudi.avro.model.HoodieWriteStat; +import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.table.HoodieTimelineArchiveLog; import java.util.LinkedHashMap; import java.util.List; @@ -36,7 +36,7 @@ public class HoodieTestCommitUtilities { */ public static org.apache.hudi.avro.model.HoodieCommitMetadata convertAndOrderCommitMetadata( HoodieCommitMetadata hoodieCommitMetadata) { - return orderCommitMetadata(HoodieTimelineArchiveLog.convertCommitMetadata(hoodieCommitMetadata)); + return orderCommitMetadata(MetadataConversionUtils.convertCommitMetadata(hoodieCommitMetadata)); } /** diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 676251e0670de..afca6a62d0d93 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -154,6 +154,23 @@ test + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + org.apache.curator + curator-client + ${zk-curator.version} + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + org.junit.jupiter @@ -195,6 +212,13 @@ junit-platform-commons test + + org.apache.curator + curator-test + ${zk-curator.version} + test + + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index a3ba00895e4a1..656b209e78042 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -22,6 +22,7 @@ import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -31,15 +32,19 @@ import org.apache.hudi.callback.util.HoodieCommitCallbackFactory; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.heartbeat.HeartbeatUtils; +import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.TableServiceType; 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.CleanerUtils; import org.apache.hudi.common.util.ClusteringUtils; @@ -101,6 +106,8 @@ public abstract class AbstractHoodieWriteClient lastCompletedTxn; /** * Create a write client, with new hudi index. @@ -124,6 +131,7 @@ public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig super(context, writeConfig, timelineService); this.metrics = new HoodieMetrics(config, config.getTableName()); this.index = createIndex(writeConfig); + this.txnManager = new TransactionManager(config, fs); } protected abstract HoodieIndex createIndex(HoodieWriteConfig writeConfig); @@ -163,24 +171,26 @@ public boolean commitStats(String instantTime, List stats, Opti public boolean commitStats(String instantTime, List stats, Option> extraMetadata, String commitActionType, Map> partitionToReplaceFileIds) { - LOG.info("Committing " + instantTime + " action " + commitActionType); // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = createTable(config, hadoopConf); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType); // Finalize write finalizeWrite(table, instantTime, stats); HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime)), + lastCompletedTxn); try { - activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + preCommit(instantTime, metadata); + commit(table, commitActionType, instantTime, metadata, stats); postCommit(table, metadata, instantTime, extraMetadata); emitCommitMetrics(instantTime, metadata, commitActionType); LOG.info("Committed " + instantTime); } catch (IOException e) { throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, e); + } finally { + this.txnManager.endTransaction(); } // callback if needed. @@ -193,6 +203,16 @@ public boolean commitStats(String instantTime, List stats, Opti return true; } + protected void commit(HoodieTable table, String commitActionType, String instantTime, HoodieCommitMetadata metadata, + List stats) throws IOException { + LOG.info("Committing " + instantTime + " action " + commitActionType); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + // Finalize write + finalizeWrite(table, instantTime, stats); + activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf); void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { @@ -210,6 +230,11 @@ void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String } } + protected void preCommit(String instantTime, HoodieCommitMetadata metadata) { + // no-op + // TODO : Conflict resolution is not supported for Flink & Java engines + } + protected void syncTableMetadata() { // no-op } @@ -227,6 +252,9 @@ protected void syncTableMetadata() { */ public void bootstrap(Option> extraMetadata) { // TODO : MULTIWRITER -> check if failed bootstrap files can be cleaned later + if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { + throw new HoodieException("Cannot bootstrap the table in multi-writer mode"); + } HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); rollbackFailedBootstrap(); table.bootstrap(context, extraMetadata); @@ -359,10 +387,19 @@ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instan * Common method containing steps to be performed before write (upsert/insert/... * @param instantTime * @param writeOperationType + * @param metaClient */ - protected void preWrite(String instantTime, WriteOperationType writeOperationType) { + protected void preWrite(String instantTime, WriteOperationType writeOperationType, + HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); - syncTableMetadata(); + this.lastCompletedTxn = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .lastInstant(); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxn); + try { + syncTableMetadata(); + } finally { + this.txnManager.endTransaction(); + } this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); } @@ -389,24 +426,25 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me // Delete the marker directory for the instant. new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - // Do an inline compaction if enabled - if (config.isInlineCompaction()) { - runAnyPendingCompactions(table); - metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); - inlineCompact(extraMetadata); - } else { - metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); - } + if (config.inlineTableServices()) { + // Do an inline compaction if enabled + if (config.inlineCompactionEnabled()) { + runAnyPendingCompactions(table); + metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); + inlineCompact(extraMetadata); + } else { + metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); + } - // Do an inline clustering if enabled - if (config.isInlineClustering()) { - runAnyPendingClustering(table); - metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"); - inlineCluster(extraMetadata); - } else { - metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false"); + // Do an inline clustering if enabled + if (config.inlineClusteringEnabled()) { + runAnyPendingClustering(table); + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"); + inlineCluster(extraMetadata); + } else { + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false"); + } } - // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); @@ -587,6 +625,20 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H * cleaned) */ public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException { + return clean(cleanInstantTime, true); + } + + /** + * Clean up any stale/old files/data lying around (either on file storage or index storage) based on the + * configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be + * cleaned). This API provides the flexibility to schedule clean instant asynchronously via + * {@link AbstractHoodieWriteClient#scheduleTableService(String, Option, TableServiceType)} and disable inline scheduling + * of clean. + */ + public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline) throws HoodieIOException { + if (scheduleInline) { + scheduleCleaningAtInstant(cleanInstantTime, Option.empty()); + } LOG.info("Cleaner started"); final Timer.Context timerContext = metrics.getCleanCtx(); LOG.info("Cleaned failed attempts if any"); @@ -663,17 +715,15 @@ private void startCommit(String instantTime, String actionType, HoodieTableMetaC /** * Schedules a new compaction instant. - * * @param extraMetadata Extra Metadata to be stored */ public Option scheduleCompaction(Option> extraMetadata) throws HoodieIOException { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + return scheduleTableService(instantTime, extraMetadata, TableServiceType.COMPACT); } /** * Schedules a new compaction instant with passed-in instant time. - * * @param instantTime Compaction Instant Time * @param extraMetadata Extra Metadata to be stored */ @@ -723,14 +773,14 @@ public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTabl /** * Get inflight time line exclude compaction and clustering. - * @param table + * @param metaClient * @return */ - private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTable table) { - HoodieTimeline inflightTimelineWithReplaceCommit = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTableMetaClient metaClient) { + HoodieTimeline inflightTimelineWithReplaceCommit = metaClient.getCommitsTimeline().filterPendingExcludingCompaction(); HoodieTimeline inflightTimelineExcludeClusteringCommit = inflightTimelineWithReplaceCommit.filter(instant -> { if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { - Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); + Option> instantPlan = ClusteringUtils.getClusteringPlan(metaClient, instant); return !instantPlan.isPresent(); } else { return true; @@ -744,7 +794,12 @@ private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieT */ public Boolean rollbackFailedWrites() { HoodieTable table = createTable(config, hadoopConf); - List instantsToRollback = getInstantsToRollback(table); + List instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy()); + rollbackFailedWrites(instantsToRollback); + return true; + } + + protected void rollbackFailedWrites(List instantsToRollback) { for (String instant : instantsToRollback) { if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { @@ -761,15 +816,14 @@ public Boolean rollbackFailedWrites() { } catch (IOException io) { LOG.error("Unable to delete heartbeat files", io); } - return true; } - private List getInstantsToRollback(HoodieTable table) { - Stream inflightInstantsStream = getInflightTimelineExcludeCompactionAndClustering(table) + protected List getInstantsToRollback(HoodieTableMetaClient metaClient, HoodieFailedWritesCleaningPolicy cleaningPolicy) { + Stream inflightInstantsStream = getInflightTimelineExcludeCompactionAndClustering(metaClient) .getReverseOrderedInstants(); - if (config.getFailedWritesCleanPolicy().isEager()) { + if (cleaningPolicy.isEager()) { return inflightInstantsStream.map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - } else if (config.getFailedWritesCleanPolicy().isLazy()) { + } else if (cleaningPolicy.isLazy()) { return inflightInstantsStream.filter(instant -> { try { return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); @@ -777,7 +831,7 @@ private List getInstantsToRollback(HoodieTable table) { throw new HoodieException("Failed to check heartbeat for instant " + instant, io); } }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - } else if (config.getFailedWritesCleanPolicy().isNever()) { + } else if (cleaningPolicy.isNever()) { return Collections.EMPTY_LIST; } else { throw new IllegalArgumentException("Invalid Failed Writes Cleaning Policy " + config.getFailedWritesCleanPolicy()); @@ -796,27 +850,27 @@ private List getInstantsToRollback(HoodieTable table) { * Performs a compaction operation on a table, serially before or after an insert/upsert action. */ protected Option inlineCompact(Option> extraMetadata) { - Option compactionInstantTimeOpt = scheduleCompaction(extraMetadata); - compactionInstantTimeOpt.ifPresent(compactionInstantTime -> { + String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime(); + Option compactionInstantTimeOpt = scheduleTableServiceInternal(compactionInstantTime, + extraMetadata, TableServiceType.COMPACT); + compactionInstantTimeOpt.ifPresent(compactInstantTime -> { // inline compaction should auto commit as the user is never given control - compact(compactionInstantTime, true); + compact(compactInstantTime, true); }); return compactionInstantTimeOpt; } /** * Schedules a new clustering instant. - * * @param extraMetadata Extra Metadata to be stored */ public Option scheduleClustering(Option> extraMetadata) throws HoodieIOException { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleClusteringAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLUSTER); } /** * Schedules a new clustering instant with passed-in instant time. - * * @param instantTime clustering Instant Time * @param extraMetadata Extra Metadata to be stored */ @@ -827,6 +881,27 @@ public boolean scheduleClusteringAtInstant(String instantTime, Option scheduleCleaning(Option> extraMetadata) throws HoodieIOException { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLEAN); + } + + /** + * Schedules a new cleaning instant with passed-in instant time. + * @param instantTime cleaning Instant Time + * @param extraMetadata Extra Metadata to be stored + */ + protected boolean scheduleCleaningAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { + LOG.info("Scheduling clustering at instant time :" + instantTime); + Option plan = createTable(config, hadoopConf) + .scheduleCleaning(context, instantTime, extraMetadata); + return plan.isPresent(); + } + /** * Ensures clustering instant is in expected state and performs clustering for the plan stored in metadata. * @@ -835,11 +910,59 @@ public boolean scheduleClusteringAtInstant(String instantTime, Option cluster(String clusteringInstant, boolean shouldComplete); + /** + * Schedule table services such as clustering, compaction & cleaning. + * + * @param extraMetadata Metadata to pass onto the scheduled service instant + * @param tableServiceType Type of table service to schedule + * @return + */ + public Option scheduleTableService(Option> extraMetadata, TableServiceType tableServiceType) { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleTableService(instantTime, extraMetadata, tableServiceType); + } + + /** + * Schedule table services such as clustering, compaction & cleaning. + * + * @param extraMetadata Metadata to pass onto the scheduled service instant + * @param tableServiceType Type of table service to schedule + * @return + */ + public Option scheduleTableService(String instantTime, Option> extraMetadata, + TableServiceType tableServiceType) { + // A lock is required to guard against race conditions between an on-going writer and scheduling a table service. + try { + this.txnManager.beginTransaction(Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, + tableServiceType.getAction(), instantTime)), Option.empty()); + LOG.info("Scheduling table service " + tableServiceType); + return scheduleTableServiceInternal(instantTime, extraMetadata, tableServiceType); + } finally { + this.txnManager.endTransaction(); + } + } + + private Option scheduleTableServiceInternal(String instantTime, Option> extraMetadata, + TableServiceType tableServiceType) { + switch (tableServiceType) { + case CLUSTER: + return scheduleClusteringAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + case COMPACT: + return scheduleCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + case CLEAN: + return scheduleCleaningAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + default: + throw new IllegalArgumentException("Invalid TableService " + tableServiceType); + } + } + /** * Executes a clustering plan on a table, serially before or after an insert/upsert action. */ protected Option inlineCluster(Option> extraMetadata) { - Option clusteringInstantOpt = scheduleClustering(extraMetadata); + String clusteringInstantTme = HoodieActiveTimeline.createNewInstantTime(); + Option clusteringInstantOpt = scheduleTableServiceInternal(clusteringInstantTme, + extraMetadata, TableServiceType.CLUSTER); clusteringInstantOpt.ifPresent(clusteringInstant -> { // inline cluster should auto commit as the user is never given control cluster(clusteringInstant, true); @@ -923,12 +1046,12 @@ public void close() { // release AsyncCleanerService AsyncCleanerService.forceShutdown(asyncCleanerService); asyncCleanerService = null; - // Stop timeline-server if running super.close(); // Calling this here releases any resources used by your index, so make sure to finish any related operations // before this point this.index.close(); this.heartbeatClient.stop(); + this.txnManager.close(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java new file mode 100644 index 0000000000000..e78a157a5fd16 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import java.io.IOException; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.client.utils.MetadataConversionUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieMetadataWrapper; +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.CommitUtils; +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hudi.common.util.Option; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; + +/** + * This class is used to hold all information used to identify how to resolve conflicts between instants. + * Since we interchange payload types between AVRO specific records and POJO's, this object serves as + * a common payload to manage these conversions. + */ +public class ConcurrentOperation { + + private WriteOperationType operationType; + private final HoodieMetadataWrapper metadataWrapper; + private final Option commitMetadataOption; + private final String actionState; + private final String actionType; + private final String instantTime; + private Set mutatedFileIds = Collections.EMPTY_SET; + + public ConcurrentOperation(HoodieInstant instant, HoodieTableMetaClient metaClient) throws IOException { + this.metadataWrapper = new HoodieMetadataWrapper(MetadataConversionUtils.createMetaWrapper(instant, metaClient)); + this.commitMetadataOption = Option.empty(); + this.actionState = instant.getState().name(); + this.actionType = instant.getAction(); + this.instantTime = instant.getTimestamp(); + init(instant); + } + + public ConcurrentOperation(HoodieInstant instant, HoodieCommitMetadata commitMetadata) { + this.commitMetadataOption = Option.of(commitMetadata); + this.metadataWrapper = new HoodieMetadataWrapper(commitMetadata); + this.actionState = instant.getState().name(); + this.actionType = instant.getAction(); + this.instantTime = instant.getTimestamp(); + init(instant); + } + + public String getInstantActionState() { + return actionState; + } + + public String getInstantActionType() { + return actionType; + } + + public String getInstantTimestamp() { + return instantTime; + } + + public WriteOperationType getOperationType() { + return operationType; + } + + public Set getMutatedFileIds() { + return mutatedFileIds; + } + + public Option getCommitMetadataOption() { + return commitMetadataOption; + } + + private void init(HoodieInstant instant) { + if (this.metadataWrapper.isAvroMetadata()) { + switch (getInstantActionType()) { + case COMPACTION_ACTION: + this.operationType = WriteOperationType.COMPACT; + this.mutatedFileIds = this.metadataWrapper.getMetadataFromTimeline().getHoodieCompactionPlan().getOperations() + .stream() + .map(op -> op.getFileId()) + .collect(Collectors.toSet()); + break; + case COMMIT_ACTION: + case DELTA_COMMIT_ACTION: + this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata() + .getPartitionToWriteStats()).keySet(); + this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata().getOperationType()); + break; + case REPLACE_COMMIT_ACTION: + if (instant.isCompleted()) { + this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord( + this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToWriteStats()).keySet(); + this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getOperationType()); + } else { + HoodieRequestedReplaceMetadata requestedReplaceMetadata = this.metadataWrapper.getMetadataFromTimeline().getHoodieRequestedReplaceMetadata(); + this.mutatedFileIds = requestedReplaceMetadata + .getClusteringPlan().getInputGroups() + .stream() + .flatMap(ig -> ig.getSlices().stream()) + .map(file -> file.getFileId()) + .collect(Collectors.toSet()); + this.operationType = WriteOperationType.CLUSTER; + } + break; + default: + throw new IllegalArgumentException("Unsupported Action Type " + getInstantActionType()); + } + } else { + switch (getInstantActionType()) { + case COMMIT_ACTION: + case DELTA_COMMIT_ACTION: + this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePaths(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats()).keySet(); + this.operationType = this.metadataWrapper.getCommitMetadata().getOperationType(); + break; + default: + throw new IllegalArgumentException("Unsupported Action Type " + getInstantActionType()); + } + } + } + + @Override + public String toString() { + return "{" + + "actionType=" + this.getInstantActionType() + + ", instantTime=" + this.getInstantTimestamp() + + ", actionState=" + this.getInstantActionState() + + '\'' + '}'; + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java new file mode 100644 index 0000000000000..d1e988adb59ae --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIMethod; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieWriteConflictException; +import org.apache.hudi.table.HoodieTable; + +import java.util.stream.Stream; + +/** + * Strategy interface for conflict resolution with multiple writers. + * Users can provide pluggable implementations for different kinds of strategies to resolve conflicts when multiple + * writers are mutating the hoodie table. + */ +public interface ConflictResolutionStrategy { + + /** + * Stream of instants to check conflicts against. + * @return + */ + Stream getCandidateInstants(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant, Option lastSuccessfulInstant); + + /** + * Implementations of this method will determine whether a conflict exists between 2 commits. + * @param thisOperation + * @param otherOperation + * @return + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation); + + /** + * Implementations of this method will determine how to resolve a conflict between 2 commits. + * @param thisOperation + * @param otherOperation + * @return + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + Option resolveConflict(HoodieTable table, + ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) throws HoodieWriteConflictException; + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java new file mode 100644 index 0000000000000..e7cad4452501b --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hudi.common.model.HoodieCommitMetadata; +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.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieWriteConflictException; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ConcurrentModificationException; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Stream; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; + +/** + * This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy}. + */ +public class SimpleConcurrentFileWritesConflictResolutionStrategy + implements ConflictResolutionStrategy { + + private static final Logger LOG = LogManager.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class); + + @Override + public Stream getCandidateInstants(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant, + Option lastSuccessfulInstant) { + + // To find which instants are conflicting, we apply the following logic + // 1. Get completed instants timeline only for commits that have happened since the last successful write. + // 2. Get any scheduled or completed compaction or clustering operations that have started and/or finished + // after the current instant. We need to check for write conflicts since they may have mutated the same files + // that are being newly created by the current write. + // NOTE that any commits from table services such as compaction, clustering or cleaning since the + // overlapping of files is handled using MVCC. + Stream completedCommitsInstantStream = activeTimeline + .getCommitsTimeline() + .filterCompletedInstants() + .findInstantsAfter(lastSuccessfulInstant.isPresent() ? lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS) + .getInstants(); + + Stream compactionAndClusteringPendingTimeline = activeTimeline + .getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, COMPACTION_ACTION)) + .findInstantsAfter(currentInstant.getTimestamp()) + .filterInflightsAndRequested() + .getInstants(); + return Stream.concat(completedCommitsInstantStream, compactionAndClusteringPendingTimeline); + } + + @Override + public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { + // TODO : UUID's can clash even for insert/insert, handle that case. + Set fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds(); + Set fileIdsSetForSecondInstant = otherOperation.getMutatedFileIds(); + Set intersection = new HashSet<>(fileIdsSetForFirstInstant); + intersection.retainAll(fileIdsSetForSecondInstant); + if (!intersection.isEmpty()) { + LOG.info("Found conflicting writes between first operation = " + thisOperation + + ", second operation = " + otherOperation + " , intersecting file ids " + intersection); + return true; + } + return false; + } + + @Override + public Option resolveConflict(HoodieTable table, + ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { + // A completed COMPACTION action eventually shows up as a COMMIT action on the timeline. + // We need to ensure we handle this during conflict resolution and not treat the commit from a + // compaction operation as a regular commit. Regular commits & deltacommits are candidates for conflict. + // Since the REPLACE action with CLUSTER operation does not support concurrent updates, we have + // to consider it as conflict if we see overlapping file ids. Once concurrent updates are + // supported for CLUSTER (https://issues.apache.org/jira/browse/HUDI-1042), + // add that to the below check so that concurrent updates do not conflict. + if (otherOperation.getOperationType() == WriteOperationType.COMPACT + && HoodieTimeline.compareTimestamps(otherOperation.getInstantTimestamp(), HoodieTimeline.LESSER_THAN, thisOperation.getInstantTimestamp())) { + return thisOperation.getCommitMetadataOption(); + } + // just abort the current write if conflicts are found + throw new HoodieWriteConflictException(new ConcurrentModificationException("Cannot resolve conflicts for overlapping writes")); + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java new file mode 100644 index 0000000000000..a6753aaa3a2ed --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.client.transaction.lock.LockManager; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Serializable; + +/** + * This class allows clients to start and end transactions. Anything done between a start and end transaction is + * guaranteed to be atomic. + */ +public class TransactionManager implements Serializable { + + private static final Logger LOG = LogManager.getLogger(TransactionManager.class); + + private final LockManager lockManager; + private Option currentTxnOwnerInstant; + private Option lastCompletedTxnOwnerInstant; + private boolean supportsOptimisticConcurrency; + + public TransactionManager(HoodieWriteConfig config, FileSystem fs) { + this.lockManager = new LockManager(config, fs); + this.supportsOptimisticConcurrency = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + } + + public synchronized void beginTransaction() { + if (supportsOptimisticConcurrency) { + LOG.info("Transaction starting without a transaction owner"); + lockManager.lock(); + LOG.info("Transaction started"); + } + } + + public synchronized void beginTransaction(Option currentTxnOwnerInstant, Option lastCompletedTxnOwnerInstant) { + if (supportsOptimisticConcurrency) { + this.lastCompletedTxnOwnerInstant = lastCompletedTxnOwnerInstant; + lockManager.setLatestCompletedWriteInstant(lastCompletedTxnOwnerInstant); + LOG.info("Latest completed transaction instant " + lastCompletedTxnOwnerInstant); + this.currentTxnOwnerInstant = currentTxnOwnerInstant; + LOG.info("Transaction starting with transaction owner " + currentTxnOwnerInstant); + lockManager.lock(); + LOG.info("Transaction started"); + } + } + + public synchronized void endTransaction() { + if (supportsOptimisticConcurrency) { + LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant); + lockManager.unlock(); + LOG.info("Transaction ended"); + this.lastCompletedTxnOwnerInstant = Option.empty(); + lockManager.resetLatestCompletedWriteInstant(); + } + } + + public void close() { + if (supportsOptimisticConcurrency) { + lockManager.close(); + LOG.info("Transaction manager closed"); + } + } + + public Option getLastCompletedTransactionOwner() { + return lastCompletedTxnOwnerInstant; + } + + public Option getCurrentTransactionOwner() { + return currentTxnOwnerInstant; + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java new file mode 100644 index 0000000000000..4b8004c03ba83 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction.lock; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP; + +import java.io.Serializable; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.lock.LockProvider; +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.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * This class wraps implementations of {@link LockProvider} and provides an easy way to manage the lifecycle of a lock. + */ +public class LockManager implements Serializable, AutoCloseable { + + private static final Logger LOG = LogManager.getLogger(LockManager.class); + private final HoodieWriteConfig writeConfig; + private final LockConfiguration lockConfiguration; + private final SerializableConfiguration hadoopConf; + private volatile LockProvider lockProvider; + // Holds the latest completed write instant to know which ones to check conflict against + private final AtomicReference> latestCompletedWriteInstant; + + public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { + this.latestCompletedWriteInstant = new AtomicReference<>(Option.empty()); + this.writeConfig = writeConfig; + this.hadoopConf = new SerializableConfiguration(fs.getConf()); + this.lockConfiguration = new LockConfiguration(writeConfig.getProps()); + } + + public void lock() { + if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { + LockProvider lockProvider = getLockProvider(); + int retryCount = 0; + boolean acquired = false; + int retries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP); + long waitTimeInMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP); + while (retryCount <= retries) { + try { + acquired = lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), TimeUnit.MILLISECONDS); + if (acquired) { + break; + } + LOG.info("Retrying to acquire lock..."); + Thread.sleep(waitTimeInMs); + retryCount++; + } catch (InterruptedException e) { + if (retryCount >= retries) { + throw new HoodieLockException("Unable to acquire lock, lock object ", e); + } + } + } + if (!acquired) { + throw new HoodieLockException("Unable to acquire lock, lock object " + lockProvider.getLock()); + } + } + } + + public void unlock() { + if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { + getLockProvider().unlock(); + } + } + + public synchronized LockProvider getLockProvider() { + // Perform lazy initialization of lock provider only if needed + if (lockProvider == null) { + LOG.info("Lock Provider " + writeConfig.getLockProviderClass()); + lockProvider = (LockProvider) ReflectionUtils.loadClass(writeConfig.getLockProviderClass(), + lockConfiguration, hadoopConf.get()); + } + return lockProvider; + } + + public void setLatestCompletedWriteInstant(Option instant) { + this.latestCompletedWriteInstant.set(instant); + } + + public void compareAndSetLatestCompletedWriteInstant(Option expected, Option newValue) { + this.latestCompletedWriteInstant.compareAndSet(expected, newValue); + } + + public AtomicReference> getLatestCompletedWriteInstant() { + return latestCompletedWriteInstant; + } + + public void resetLatestCompletedWriteInstant() { + this.latestCompletedWriteInstant.set(Option.empty()); + } + + @Override + public void close() { + closeQuietly(); + } + + private void closeQuietly() { + try { + if (lockProvider != null) { + lockProvider.close(); + LOG.info("Released connection created for acquiring lock"); + lockProvider = null; + } + } catch (Exception e) { + LOG.error("Unable to close and release connection created for acquiring lock", e); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java new file mode 100644 index 0000000000000..60336c53e04f3 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction.lock; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.CuratorFrameworkState; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations + * using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock. + */ +@NotThreadSafe +public class ZookeeperBasedLockProvider implements LockProvider { + + private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); + + private final CuratorFramework curatorFrameworkClient; + private volatile InterProcessMutex lock = null; + protected LockConfiguration lockConfiguration; + + public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = CuratorFrameworkFactory.builder() + .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)) + .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP), + 5000, lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) + .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS)) + .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS)) + .build(); + this.curatorFrameworkClient.start(); + } + + // Only used for testing + public ZookeeperBasedLockProvider( + final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = curatorFrameworkClient; + synchronized (this.curatorFrameworkClient) { + if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) { + this.curatorFrameworkClient.start(); + } + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (HoodieLockException e) { + throw e; + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } + return lock != null && lock.isAcquiredInThisProcess(); + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString())); + if (lock == null || !lock.isAcquiredInThisProcess()) { + return; + } + lock.release(); + lock = null; + LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + @Override + public void close() { + try { + if (lock != null) { + lock.release(); + lock = null; + } + this.curatorFrameworkClient.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + @Override + public InterProcessMutex getLock() { + return this.lock; + } + + private void acquireLock(long time, TimeUnit unit) throws Exception { + ValidationUtils.checkArgument(this.lock == null, generateLogStatement(LockState.ALREADY_ACQUIRED, generateLogSuffixString())); + InterProcessMutex newLock = new InterProcessMutex( + this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/" + + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP)); + boolean acquired = newLock.acquire(time, unit); + if (!acquired) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString())); + } + if (newLock.isAcquiredInThisProcess()) { + lock = newLock; + } else { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString())); + } + } + + private void checkRequiredProps(final LockConfiguration config) { + ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECT_URL_PROP) != null); + ValidationUtils.checkArgument(config.getConfig().getString(ZK_BASE_PATH_PROP) != null); + ValidationUtils.checkArgument(config.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP) != null); + ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP) != null); + ValidationUtils.checkArgument(config.getConfig().getString(ZK_LOCK_KEY_PROP) != null); + } + + private String generateLogSuffixString() { + String zkBasePath = this.lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP); + String lockKey = this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP); + return StringUtils.join("ZkBasePath = ", zkBasePath, ", lock key = ", lockKey); + } + + protected String generateLogStatement(LockState state, String suffix) { + return StringUtils.join(state.name(), " lock at", suffix); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java new file mode 100644 index 0000000000000..49b93182a8c67 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.client.ReplaceArchivalHelper; +import org.apache.hudi.common.model.ActionType; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieRollingStatMetadata; +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; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CompactionUtils; + +/** + * Helper class to convert between different action related payloads and {@link HoodieArchivedMetaEntry}. + */ +public class MetadataConversionUtils { + + public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInstant, HoodieTableMetaClient metaClient) throws IOException { + HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry(); + archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp()); + archivedMetaWrapper.setActionState(hoodieInstant.getState().name()); + switch (hoodieInstant.getAction()) { + case HoodieTimeline.CLEAN_ACTION: { + if (hoodieInstant.isCompleted()) { + archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, hoodieInstant)); + } else { + archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, hoodieInstant)); + } + archivedMetaWrapper.setActionType(ActionType.clean.name()); + break; + } + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.DELTA_COMMIT_ACTION: { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); + archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata)); + archivedMetaWrapper.setActionType(ActionType.commit.name()); + break; + } + case HoodieTimeline.REPLACE_COMMIT_ACTION: { + if (hoodieInstant.isCompleted()) { + HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata + .fromBytes(metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); + archivedMetaWrapper.setHoodieReplaceCommitMetadata(ReplaceArchivalHelper.convertReplaceCommitMetadata(replaceCommitMetadata)); + } else { + HoodieRequestedReplaceMetadata requestedReplaceMetadata = + ClusteringUtils.getRequestedReplaceMetadata(metaClient, hoodieInstant).get(); + archivedMetaWrapper.setHoodieRequestedReplaceMetadata(requestedReplaceMetadata); + } + archivedMetaWrapper.setActionType(ActionType.replacecommit.name()); + break; + } + case HoodieTimeline.ROLLBACK_ACTION: { + archivedMetaWrapper.setHoodieRollbackMetadata(TimelineMetadataUtils.deserializeAvroMetadata( + metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); + archivedMetaWrapper.setActionType(ActionType.rollback.name()); + break; + } + case HoodieTimeline.SAVEPOINT_ACTION: { + archivedMetaWrapper.setHoodieSavePointMetadata(TimelineMetadataUtils.deserializeAvroMetadata( + metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class)); + archivedMetaWrapper.setActionType(ActionType.savepoint.name()); + break; + } + case HoodieTimeline.COMPACTION_ACTION: { + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, hoodieInstant.getTimestamp()); + archivedMetaWrapper.setHoodieCompactionPlan(plan); + archivedMetaWrapper.setActionType(ActionType.compaction.name()); + break; + } + default: { + throw new UnsupportedOperationException("Action not fully supported yet"); + } + } + return archivedMetaWrapper; + } + + public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInstant, + HoodieCommitMetadata hoodieCommitMetadata) { + HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry(); + archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp()); + archivedMetaWrapper.setActionState(hoodieInstant.getState().name()); + archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(hoodieCommitMetadata)); + archivedMetaWrapper.setActionType(ActionType.commit.name()); + return archivedMetaWrapper; + } + + public static org.apache.hudi.avro.model.HoodieCommitMetadata convertCommitMetadata( + HoodieCommitMetadata hoodieCommitMetadata) { + ObjectMapper mapper = new ObjectMapper(); + // Need this to ignore other public get() methods + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData = + mapper.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class); + if (hoodieCommitMetadata.getCompacted()) { + avroMetaData.setOperationType(WriteOperationType.COMPACT.name()); + } + // Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer + avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, ""); + return avroMetaData; + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java new file mode 100644 index 0000000000000..d4ba4fb0ddadd --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.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.client.utils; + +import org.apache.hudi.client.transaction.ConflictResolutionStrategy; +import org.apache.hudi.client.transaction.ConcurrentOperation; +import org.apache.hudi.common.model.HoodieCommitMetadata; +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.exception.HoodieWriteConflictException; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import java.io.IOException; +import java.util.stream.Stream; + +public class TransactionUtils { + + private static final Logger LOG = LogManager.getLogger(TransactionUtils.class); + + /** + * Resolve any write conflicts when committing data. + * @param table + * @param currentTxnOwnerInstant + * @param thisCommitMetadata + * @param config + * @param lastCompletedTxnOwnerInstant + * @return + * @throws HoodieWriteConflictException + */ + public static Option resolveWriteConflictIfAny(final HoodieTable table, final Option currentTxnOwnerInstant, + final Option thisCommitMetadata, final HoodieWriteConfig config, Option lastCompletedTxnOwnerInstant) throws HoodieWriteConflictException { + if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { + ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy(); + Stream instantStream = resolutionStrategy.getCandidateInstants(table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant); + final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.get()); + instantStream.forEach(instant -> { + try { + ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient()); + if (resolutionStrategy.hasConflict(thisOperation, otherOperation)) { + LOG.info("Conflict encountered between current instant = " + thisOperation + " and instant = " + + otherOperation + ", attempting to resolve it..."); + resolutionStrategy.resolveConflict(table, thisOperation, otherOperation); + } + } catch (IOException io) { + throw new HoodieWriteConflictException("Unable to resolve conflict, if present", io); + } + }); + LOG.info("Successfully resolved conflicts, if any"); + return thisOperation.getCommitMetadataOption(); + } + return thisCommitMetadata; + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index e05195b9d7767..43af00a139b8a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -43,7 +43,6 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy"; public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic"; public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async"; - // Turn on inline compaction - after fw delta commits a inline compaction will be run public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline"; // Run a compaction every N delta commits diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java new file mode 100644 index 0000000000000..1f0328e2fe000 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java @@ -0,0 +1,191 @@ +/* + * 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.config; + +import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; +import org.apache.hudi.client.transaction.ConflictResolutionStrategy; +import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; +import org.apache.hudi.common.config.DefaultHoodieConfig; +import org.apache.hudi.common.lock.LockProvider; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + + +/** + * Hoodie Configs for Locks. + */ +public class HoodieLockConfig extends DefaultHoodieConfig { + + // Pluggable type of lock provider + public static final String LOCK_PROVIDER_CLASS_PROP = LOCK_PREFIX + "provider"; + public static final String DEFAULT_LOCK_PROVIDER_CLASS = ZookeeperBasedLockProvider.class.getName(); + // Pluggable strategies to use when resolving conflicts + public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP = + LOCK_PREFIX + "conflict.resolution.strategy"; + public static final String DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS = + SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName(); + + private HoodieLockConfig(Properties props) { + super(props); + } + + public static HoodieLockConfig.Builder newBuilder() { + return new HoodieLockConfig.Builder(); + } + + public static class Builder { + + private final Properties props = new Properties(); + + public HoodieLockConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.props.load(reader); + return this; + } + } + + public HoodieLockConfig.Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public HoodieLockConfig.Builder withLockProvider(Class lockProvider) { + props.setProperty(LOCK_PROVIDER_CLASS_PROP, lockProvider.getName()); + return this; + } + + public HoodieLockConfig.Builder withHiveDatabaseName(String databaseName) { + props.setProperty(HIVE_DATABASE_NAME_PROP, databaseName); + return this; + } + + public HoodieLockConfig.Builder withHiveTableName(String tableName) { + props.setProperty(HIVE_TABLE_NAME_PROP, tableName); + return this; + } + + public HoodieLockConfig.Builder withZkQuorum(String zkQuorum) { + props.setProperty(ZK_CONNECT_URL_PROP, zkQuorum); + return this; + } + + public HoodieLockConfig.Builder withZkBasePath(String zkBasePath) { + props.setProperty(ZK_BASE_PATH_PROP, zkBasePath); + return this; + } + + public HoodieLockConfig.Builder withZkPort(String zkPort) { + props.setProperty(ZK_PORT_PROP, zkPort); + return this; + } + + public HoodieLockConfig.Builder withZkLockKey(String zkLockKey) { + props.setProperty(ZK_LOCK_KEY_PROP, zkLockKey); + return this; + } + + public HoodieLockConfig.Builder withZkConnectionTimeoutInMs(Long connectionTimeoutInMs) { + props.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(connectionTimeoutInMs)); + return this; + } + + public HoodieLockConfig.Builder withZkSessionTimeoutInMs(Long sessionTimeoutInMs) { + props.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(sessionTimeoutInMs)); + return this; + } + + public HoodieLockConfig.Builder withNumRetries(int numRetries) { + props.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, String.valueOf(numRetries)); + return this; + } + + public HoodieLockConfig.Builder withRetryWaitTimeInMillis(Long retryWaitTimeInMillis) { + props.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryWaitTimeInMillis)); + return this; + } + + public HoodieLockConfig.Builder withClientNumRetries(int clientNumRetries) { + props.setProperty(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, String.valueOf(clientNumRetries)); + return this; + } + + public HoodieLockConfig.Builder withClientRetryWaitTimeInMillis(Long clientRetryWaitTimeInMillis) { + props.setProperty(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(clientRetryWaitTimeInMillis)); + return this; + } + + public HoodieLockConfig.Builder withLockWaitTimeInMillis(Long waitTimeInMillis) { + props.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(waitTimeInMillis)); + return this; + } + + public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutionStrategy conflictResolutionStrategy) { + props.setProperty(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, conflictResolutionStrategy.getClass().getName()); + return this; + } + + public HoodieLockConfig build() { + HoodieLockConfig config = new HoodieLockConfig(props); + setDefaultOnCondition(props, !props.containsKey(LOCK_PROVIDER_CLASS_PROP), + LOCK_PROVIDER_CLASS_PROP, DEFAULT_LOCK_PROVIDER_CLASS); + setDefaultOnCondition(props, !props.containsKey(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP), + WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS); + setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_NUM_RETRIES_PROP), + LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES); + setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP), + LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS); + setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP), + LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES); + setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP), + LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS); + setDefaultOnCondition(props, !props.containsKey(ZK_CONNECTION_TIMEOUT_MS_PROP), + ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS)); + setDefaultOnCondition(props, !props.containsKey(ZK_SESSION_TIMEOUT_MS_PROP), + ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS)); + setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), + LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS)); + return config; + } + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 4e493e4432f51..f7ad3a0faca1b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -18,18 +18,23 @@ package org.apache.hudi.config; +import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.transaction.ConflictResolutionStrategy; import org.apache.hudi.common.config.DefaultHoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.SimpleAvroKeyGenerator; @@ -37,12 +42,9 @@ import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; - -import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import javax.annotation.concurrent.Immutable; - import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -55,6 +57,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP; /** * Class storing configs for the HoodieWriteClient. @@ -148,6 +152,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = "hoodie.client.heartbeat.tolerable.misses"; public static final Integer DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = 2; + // Enable different concurrency support + public static final String WRITE_CONCURRENCY_MODE_PROP = + "hoodie.write.concurrency.mode"; + public static final String DEFAULT_WRITE_CONCURRENCY_MODE = WriteConcurrencyMode.SINGLE_WRITER.name(); /** * HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow @@ -427,7 +435,7 @@ public boolean incrementalCleanerModeEnabled() { return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE)); } - public boolean isInlineCompaction() { + public boolean inlineCompactionEnabled() { return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP)); } @@ -459,7 +467,7 @@ public Boolean getCompactionReverseLogReadEnabled() { return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP)); } - public boolean isInlineClustering() { + public boolean inlineClusteringEnabled() { return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_PROP)); } @@ -469,7 +477,7 @@ public boolean isAsyncClusteringEnabled() { public boolean isClusteringEnabled() { // TODO: future support async clustering - return isInlineClustering() || isAsyncClusteringEnabled(); + return inlineClusteringEnabled() || isAsyncClusteringEnabled(); } public int getInlineClusterMaxCommits() { @@ -939,10 +947,6 @@ public Long getMaxMemoryPerPartitionMerge() { return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP)); } - public int getMetadataCleanerCommitsRetained() { - return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP)); - } - public Long getHoodieClientHeartbeatIntervalInMs() { return Long.valueOf(props.getProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP)); } @@ -982,6 +986,43 @@ public int getMetadataMinCommitsToKeep() { return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP)); } + public int getMetadataCleanerCommitsRetained() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP)); + } + + /** + * Hoodie Client Lock Configs. + * @return + */ + + public String getLockProviderClass() { + return props.getProperty(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP); + } + + public String getLockHiveDatabaseName() { + return props.getProperty(HIVE_DATABASE_NAME_PROP); + } + + public String getLockHiveTableName() { + return props.getProperty(HIVE_TABLE_NAME_PROP); + } + + public ConflictResolutionStrategy getWriteConflictResolutionStrategy() { + return ReflectionUtils.loadClass(props.getProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP)); + } + + public Long getLockAcquireWaitTimeoutInMs() { + return Long.valueOf(props.getProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP)); + } + + public WriteConcurrencyMode getWriteConcurrencyMode() { + return WriteConcurrencyMode.fromValue(props.getProperty(WRITE_CONCURRENCY_MODE_PROP)); + } + + public Boolean inlineTableServices() { + return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean(); + } + public static class Builder { protected final Properties props = new Properties(); @@ -998,6 +1039,7 @@ public static class Builder { private boolean isCallbackConfigSet = false; private boolean isPayloadConfigSet = false; private boolean isMetadataConfigSet = false; + private boolean isLockConfigSet = false; public Builder withEngineType(EngineType engineType) { this.engineType = engineType; @@ -1141,6 +1183,12 @@ public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) { return this; } + public Builder withLockConfig(HoodieLockConfig lockConfig) { + props.putAll(lockConfig.getProps()); + isLockConfigSet = true; + return this; + } + public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) { props.putAll(metricsConfig.getProps()); isMetricsConfigSet = true; @@ -1254,6 +1302,11 @@ public Builder withHeartbeatTolerableMisses(Integer heartbeatTolerableMisses) { return this; } + public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) { + props.setProperty(WRITE_CONCURRENCY_MODE_PROP, concurrencyMode.value()); + return this; + } + public Builder withProperties(Properties properties) { this.props.putAll(properties); return this; @@ -1314,6 +1367,8 @@ protected void setDefaults() { CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS)); setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP), CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES)); + setDefaultOnCondition(props, !props.containsKey(WRITE_CONCURRENCY_MODE_PROP), + WRITE_CONCURRENCY_MODE_PROP, DEFAULT_WRITE_CONCURRENCY_MODE); // Make sure the props is propagated setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build()); @@ -1336,6 +1391,8 @@ protected void setDefaults() { HoodiePayloadConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isMetadataConfigSet, HoodieMetadataConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isLockConfigSet, + HoodieLockConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !props.containsKey(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION), EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION); @@ -1349,6 +1406,11 @@ private void validate() { // Ensure Layout Version is good new TimelineLayoutVersion(Integer.parseInt(layoutVersion)); Objects.requireNonNull(props.getProperty(BASE_PATH_PROP)); + if (props.getProperty(WRITE_CONCURRENCY_MODE_PROP) + .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())) { + ValidationUtils.checkArgument(props.getProperty(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP) + != HoodieFailedWritesCleaningPolicy.EAGER.name()); + } } public HoodieWriteConfig build() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index dbd678f2b5f0c..dfd4c4c0457db 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -36,6 +36,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.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -89,7 +90,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta protected SerializableConfiguration hadoopConf; protected final transient HoodieEngineContext engineContext; - protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { + protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, + HoodieEngineContext engineContext) { this.datasetWriteConfig = writeConfig; this.engineContext = engineContext; this.hadoopConf = new SerializableConfiguration(hadoopConf); @@ -101,7 +103,7 @@ protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteC // Inline compaction and auto clean is required as we dont expose this table outside ValidationUtils.checkArgument(!this.metadataWriteConfig.isAutoClean(), "Cleaning is controlled internally for Metadata table."); - ValidationUtils.checkArgument(!this.metadataWriteConfig.isInlineCompaction(), "Compaction is controlled internally for metadata table."); + ValidationUtils.checkArgument(!this.metadataWriteConfig.inlineCompactionEnabled(), "Compaction is controlled internally for metadata table."); // Metadata Table cannot have metadata listing turned on. (infinite loop, much?) ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(), "Auto commit is required for Metadata Table"); ValidationUtils.checkArgument(!this.metadataWriteConfig.useFileListingMetadata(), "File listing cannot be used for Metadata Table"); @@ -143,6 +145,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withMaxConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getMaxConsistencyCheckIntervalMs()) .withMaxConsistencyChecks(writeConfig.getConsistencyGuardConfig().getMaxConsistencyChecks()) .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.SINGLE_WRITER) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .withAutoCommit(true) .withAvroSchemaValidate(true) 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 74ffdfc653173..15253f668a389 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 @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -394,6 +395,19 @@ public abstract Option scheduleClustering(HoodieEngineCont */ public abstract void rollbackBootstrap(HoodieEngineContext context, String instantTime); + + /** + * Schedule cleaning for the instant time. + * + * @param context HoodieEngineContext + * @param instantTime Instant Time for scheduling cleaning + * @param extraMetadata additional metadata to write into plan + * @return HoodieCleanerPlan, if there is anything to clean. + */ + public abstract Option scheduleCleaning(HoodieEngineContext context, + String instantTime, + Option> extraMetadata); + /** * Executes a new clean action. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index 8efd3a2268921..09df62c91fd29 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -18,24 +18,17 @@ package org.apache.hudi.table; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.ReplaceArchivalHelper; +import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; -import org.apache.hudi.common.model.HoodieRollingStatMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; @@ -46,12 +39,9 @@ import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.TableFileSystemView; -import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -59,7 +49,6 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.metadata.HoodieTableMetadata; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -378,73 +367,6 @@ private void writeToFile(Schema wrapperSchema, List records) thro private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant) throws IOException { - HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry(); - archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp()); - archivedMetaWrapper.setActionState(hoodieInstant.getState().name()); - switch (hoodieInstant.getAction()) { - case HoodieTimeline.CLEAN_ACTION: { - if (hoodieInstant.isCompleted()) { - archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, hoodieInstant)); - } else { - archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, hoodieInstant)); - } - archivedMetaWrapper.setActionType(ActionType.clean.name()); - break; - } - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.DELTA_COMMIT_ACTION: { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); - archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata)); - archivedMetaWrapper.setActionType(ActionType.commit.name()); - break; - } - case HoodieTimeline.REPLACE_COMMIT_ACTION: { - if (hoodieInstant.isRequested()) { - archivedMetaWrapper.setHoodieRequestedReplaceMetadata( - TimelineMetadataUtils.deserializeRequestedReplaceMetadata(commitTimeline.getInstantDetails(hoodieInstant).get())); - } else if (hoodieInstant.isCompleted()) { - HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); - archivedMetaWrapper.setHoodieReplaceCommitMetadata(ReplaceArchivalHelper.convertReplaceCommitMetadata(replaceCommitMetadata)); - } - archivedMetaWrapper.setActionType(ActionType.replacecommit.name()); - break; - } - case HoodieTimeline.ROLLBACK_ACTION: { - archivedMetaWrapper.setHoodieRollbackMetadata(TimelineMetadataUtils.deserializeAvroMetadata( - commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); - archivedMetaWrapper.setActionType(ActionType.rollback.name()); - break; - } - case HoodieTimeline.SAVEPOINT_ACTION: { - archivedMetaWrapper.setHoodieSavePointMetadata(TimelineMetadataUtils.deserializeAvroMetadata( - commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class)); - archivedMetaWrapper.setActionType(ActionType.savepoint.name()); - break; - } - case HoodieTimeline.COMPACTION_ACTION: { - HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, hoodieInstant.getTimestamp()); - archivedMetaWrapper.setHoodieCompactionPlan(plan); - archivedMetaWrapper.setActionType(ActionType.compaction.name()); - break; - } - default: { - throw new UnsupportedOperationException("Action not fully supported yet"); - } - } - return archivedMetaWrapper; - } - - public static org.apache.hudi.avro.model.HoodieCommitMetadata convertCommitMetadata( - HoodieCommitMetadata hoodieCommitMetadata) { - ObjectMapper mapper = new ObjectMapper(); - // Need this to ignore other public get() methods - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData = - mapper.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class); - // Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer - avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, ""); - return avroMetaData; + return MetadataConversionUtils.createMetaWrapper(hoodieInstant, metaClient); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java index 5b37820071a1e..acc3cdc6793ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java @@ -20,23 +20,17 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hudi.avro.model.HoodieActionInstant; -import org.apache.hudi.avro.model.HoodieCleanFileInfo; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CleanerUtils; -import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; @@ -46,8 +40,8 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; public abstract class BaseCleanActionExecutor extends BaseActionExecutor { @@ -59,42 +53,6 @@ public BaseCleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig co super(context, config, table, instantTime); } - /** - * Generates List of files to be cleaned. - * - * @param context HoodieEngineContext - * @return Cleaner Plan - */ - HoodieCleanerPlan requestClean(HoodieEngineContext context) { - try { - CleanPlanner planner = new CleanPlanner<>(context, table, config); - Option earliestInstant = planner.getEarliestCommitToRetain(); - List partitionsToClean = planner.getPartitionPathsToClean(earliestInstant); - - if (partitionsToClean.isEmpty()) { - LOG.info("Nothing to clean here."); - return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build(); - } - LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy()); - int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); - LOG.info("Using cleanerParallelism: " + cleanerParallelism); - - context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned"); - - Map> cleanOps = context - .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism) - .stream() - .collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue()))); - - return new HoodieCleanerPlan(earliestInstant - .map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null), - config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(), - CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps); - } catch (IOException e) { - throw new HoodieIOException("Failed to schedule clean operation", e); - } - } - protected static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException { Path deletePath = new Path(deletePathStr); LOG.debug("Working on delete path :" + deletePath); @@ -118,40 +76,13 @@ protected static Boolean deleteFileAndGetResult(FileSystem fs, String deletePath */ abstract List clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan); - /** - * Creates a Cleaner plan if there are files to be cleaned and stores them in instant file. - * Cleaner Plan contains absolute file paths. - * - * @param startCleanTime Cleaner Instant Time - * @return Cleaner Plan if generated - */ - Option requestClean(String startCleanTime) { - final HoodieCleanerPlan cleanerPlan = requestClean(context); - if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) - && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty() - && cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) { - // Only create cleaner plan which does some work - final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime); - // Save to both aux and timeline folder - try { - table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); - LOG.info("Requesting Cleaning with instant time " + cleanInstant); - } catch (IOException e) { - LOG.error("Got exception when saving cleaner requested file", e); - throw new HoodieIOException(e.getMessage(), e); - } - return Option.of(cleanerPlan); - } - return Option.empty(); - } - /** * Executes the Cleaner plan stored in the instant metadata. */ - void runPendingClean(HoodieTable table, HoodieInstant cleanInstant) { + HoodieCleanMetadata runPendingClean(HoodieTable table, HoodieInstant cleanInstant) { try { HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant); - runClean(table, cleanInstant, cleanerPlan); + return runClean(table, cleanInstant, cleanerPlan); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -195,6 +126,7 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan @Override public HoodieCleanMetadata execute() { + List cleanMetadataList = new ArrayList<>(); // If there are inflight(failed) or previously requested clean operation, first perform them List pendingCleanInstants = table.getCleanTimeline() .filterInflightsAndRequested().getInstants().collect(Collectors.toList()); @@ -202,23 +134,16 @@ public HoodieCleanMetadata execute() { pendingCleanInstants.forEach(hoodieInstant -> { LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant); try { - runPendingClean(table, hoodieInstant); + cleanMetadataList.add(runPendingClean(table, hoodieInstant)); } catch (Exception e) { LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e); } }); table.getMetaClient().reloadActiveTimeline(); } - - // Plan and execute a new clean action - Option cleanerPlanOpt = requestClean(instantTime); - if (cleanerPlanOpt.isPresent()) { - table.getMetaClient().reloadActiveTimeline(); - HoodieCleanerPlan cleanerPlan = cleanerPlanOpt.get(); - if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()) { - return runClean(table, HoodieTimeline.getCleanRequestedInstant(instantTime), cleanerPlan); - } - } - return null; + // return the last clean metadata for now + // TODO (NA) : Clean only the earliest pending clean just like how we do for other table services + // This requires the BaseCleanActionExecutor to be refactored as BaseCommitActionExecutor + return cleanMetadataList.size() > 0 ? cleanMetadataList.get(cleanMetadataList.size() - 1) : null; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanPlanActionExecutor.java new file mode 100644 index 0000000000000..fc0c000a6a864 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanPlanActionExecutor.java @@ -0,0 +1,132 @@ +/* + * 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.clean; + +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanFileInfo; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.CollectionUtils; +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.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public abstract class BaseCleanPlanActionExecutor extends BaseActionExecutor> { + + private static final Logger LOG = LogManager.getLogger(CleanPlanner.class); + + private final Option> extraMetadata; + + public BaseCleanPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime); + this.extraMetadata = extraMetadata; + } + + protected abstract Option createCleanerPlan(); + + /** + * Generates List of files to be cleaned. + * + * @param context HoodieEngineContext + * @return Cleaner Plan + */ + HoodieCleanerPlan requestClean(HoodieEngineContext context) { + try { + CleanPlanner planner = new CleanPlanner<>(context, table, config); + Option earliestInstant = planner.getEarliestCommitToRetain(); + List partitionsToClean = planner.getPartitionPathsToClean(earliestInstant); + + if (partitionsToClean.isEmpty()) { + LOG.info("Nothing to clean here. It is already clean"); + return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build(); + } + LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy()); + int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); + LOG.info("Using cleanerParallelism: " + cleanerParallelism); + + context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned"); + + Map> cleanOps = context + .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism) + .stream() + .collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue()))); + + return new HoodieCleanerPlan(earliestInstant + .map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null), + config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(), + CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps); + } catch (IOException e) { + throw new HoodieIOException("Failed to schedule clean operation", e); + } + } + + /** + * Creates a Cleaner plan if there are files to be cleaned and stores them in instant file. + * Cleaner Plan contains absolute file paths. + * + * @param startCleanTime Cleaner Instant Time + * @return Cleaner Plan if generated + */ + protected Option requestClean(String startCleanTime) { + final HoodieCleanerPlan cleanerPlan = requestClean(context); + if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) + && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty() + && cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) { + // Only create cleaner plan which does some work + final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime); + // Save to both aux and timeline folder + try { + table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); + LOG.info("Requesting Cleaning with instant time " + cleanInstant); + } catch (IOException e) { + LOG.error("Got exception when saving cleaner requested file", e); + throw new HoodieIOException(e.getMessage(), e); + } + return Option.of(cleanerPlan); + } + return Option.empty(); + } + + @Override + public Option execute() { + // Plan a new clean action + return requestClean(instantTime); + } + +} 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 4a8ddf5c57997..ffb403b213f79 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,6 +19,8 @@ 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; @@ -29,6 +31,7 @@ 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.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -58,6 +61,8 @@ public abstract class BaseCommitActionExecutor> 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, @@ -66,6 +71,10 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c 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 = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .lastInstant(); } public abstract HoodieWriteMetadata execute(I inputRecords); @@ -117,12 +126,24 @@ protected String getCommitActionType() { protected void commitOnAutoCommit(HoodieWriteMetadata result) { if (config.shouldAutoCommit()) { LOG.info("Auto commit enabled: Committing " + instantTime); - commit(extraMetadata, result); + 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); + 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); /** @@ -140,6 +161,10 @@ protected void finalizeWrite(String instantTime, List stats, Ho } } + protected void syncTableMetadata() { + // No Op + } + /** * By default, return the writer schema in Write Config for storing in commit. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java index 37efc0eb0c374..b744a7e4bb0a1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java @@ -53,22 +53,24 @@ public BaseScheduleCompactionActionExecutor(HoodieEngineContext context, @Override public Option execute() { - // if there are inflight writes, their instantTime must not be less than that of compaction instant time - table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() - .ifPresent(earliestInflight -> ValidationUtils.checkArgument( - HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), - "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight - + ", Compaction scheduled at " + instantTime)); - - // Committed and pending compaction instants should have strictly lower timestamps - List conflictingInstants = table.getActiveTimeline() - .getWriteTimeline().getInstants() - .filter(instant -> HoodieTimeline.compareTimestamps( - instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) - .collect(Collectors.toList()); - ValidationUtils.checkArgument(conflictingInstants.isEmpty(), - "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :" - + conflictingInstants); + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && !config.getFailedWritesCleanPolicy().isLazy()) { + // if there are inflight writes, their instantTime must not be less than that of compaction instant time + table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() + .ifPresent(earliestInflight -> ValidationUtils.checkArgument( + HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), + "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight + + ", Compaction scheduled at " + instantTime)); + // Committed and pending compaction instants should have strictly lower timestamps + List conflictingInstants = table.getActiveTimeline() + .getWriteTimeline().getInstants() + .filter(instant -> HoodieTimeline.compareTimestamps( + instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) + .collect(Collectors.toList()); + ValidationUtils.checkArgument(conflictingInstants.isEmpty(), + "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :" + + conflictingInstants); + } HoodieCompactionPlan plan = scheduleCompaction(); if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java index 7476a1630e247..67bc1c20986fd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java @@ -78,6 +78,12 @@ public abstract class AbstractUpgradeDowngrade { public abstract void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config, HoodieEngineContext context, String instantTime); + public boolean needsUpgradeOrDowngrade(HoodieTableVersion toVersion) { + HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion(); + // Ensure no inflight commits & versions are same + return toVersion.versionCode() != fromVersion.versionCode(); + } + protected AbstractUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) { this.metaClient = metaClient; this.config = config; @@ -90,7 +96,7 @@ protected AbstractUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWrite protected void run(HoodieTableVersion toVersion, String instantTime) throws IOException { // Fetch version from property file and current version HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion(); - if (toVersion.versionCode() == fromVersion.versionCode()) { + if (!needsUpgradeOrDowngrade(toVersion)) { return; } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java new file mode 100644 index 0000000000000..1a923650d8409 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieLockException; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; + +/** + * This lock provider is used for testing purposes only. It provides a simple file system based lock using HDFS atomic + * create operation. This lock does not support cleaning/expiring the lock after a failed write hence cannot be used + * in production environments. + */ +public class FileSystemBasedLockProviderTestClass implements LockProvider { + + private static final String LOCK_NAME = "acquired"; + + private String lockPath; + private FileSystem fs; + protected LockConfiguration lockConfiguration; + + public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfiguration, final Configuration configuration) { + try { + this.lockConfiguration = lockConfiguration; + this.lockPath = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP); + this.fs = FileSystem.get(configuration); + } catch (IOException io) { + throw new HoodieIOException("Unable to create file systems", io); + } + } + + public void acquireLock() { + try { + fs.create(new Path(lockPath + "/" + LOCK_NAME)).close(); + } catch (IOException e) { + throw new HoodieIOException("Failed to acquire lock", e); + } + } + + @Override + public void close() { + try { + fs.close(); + } catch (Exception e) { + e.printStackTrace(); + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + try { + int numRetries = 0; + while (fs.exists(new Path(lockPath + "/" + LOCK_NAME)) + && (numRetries <= lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) { + Thread.sleep(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP)); + } + acquireLock(); + return true; + } catch (IOException | InterruptedException e) { + throw new HoodieLockException("Failed to acquire lock", e); + } + } + + @Override + public void unlock() { + try { + if (fs.exists(new Path(lockPath + "/" + LOCK_NAME))) { + fs.delete(new Path(lockPath + "/" + LOCK_NAME), true); + } + } catch (IOException io) { + throw new HoodieIOException("Unable to delete lock on disk", io); + } + } + + @Override + public String getLock() { + try { + return fs.listStatus(new Path(lockPath))[0].getPath().toString(); + } catch (Exception e) { + throw new HoodieLockException("Failed to retrieve lock status from lock path " + lockPath); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java new file mode 100644 index 0000000000000..9483d61fb57ce --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -0,0 +1,420 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieCompactionOperation; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +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.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieWriteConflictException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestSimpleConcurrentFileWritesConflictResolutionStrategy extends HoodieCommonTestHarness { + + @BeforeEach + public void init() throws IOException { + initMetaClient(); + } + + @Test + public void testNoConcurrentWrites() throws Exception { + String newInstantTime = HoodieTestTable.makeNewCommitTime(); + createCommit(newInstantTime); + // consider commits before this are all successful + + Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); + newInstantTime = HoodieTestTable.makeNewCommitTime(); + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant); + Assertions.assertTrue(candidateInstants.count() == 0); + } + + @Test + public void testConcurrentWrites() throws Exception { + String newInstantTime = HoodieTestTable.makeNewCommitTime(); + createCommit(newInstantTime); + // consider commits before this are all successful + // writer 1 + createInflightCommit(HoodieTestTable.makeNewCommitTime()); + // writer 2 + createInflightCommit(HoodieTestTable.makeNewCommitTime()); + Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); + newInstantTime = HoodieTestTable.makeNewCommitTime(); + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant); + Assertions.assertTrue(candidateInstants.count() == 0); + } + + @Test + public void testConcurrentWritesWithInterleavingSuccesssfulCommit() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + // writer 2 starts and finishes + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCommit(newInstantTime); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with writer 2 + Assertions.assertTrue(candidateInstants.size() == 1); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, writer 1 and writer 2 should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @Test + public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + // compaction 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCompactionRequested(newInstantTime); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with scheduled compaction plan 1 + Assertions.assertTrue(candidateInstants.size() == 1); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @Test + public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + // compaction 1 gets scheduled and finishes + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCompaction(newInstantTime); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with compaction 1 + Assertions.assertTrue(candidateInstants.size() == 1); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @Test + public void testConcurrentWriteAndCompactionScheduledEarlier() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + // compaction 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCompaction(newInstantTime); + // consider commits before this are all successful + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 should not conflict with an earlier scheduled compaction 1 with the same file ids + Assertions.assertTrue(candidateInstants.size() == 0); + } + + @Test + public void testConcurrentWritesWithInterleavingScheduledCluster() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + // clustering 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createReplaceRequested(newInstantTime); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with scheduled compaction plan 1 + Assertions.assertTrue(candidateInstants.size() == 1); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @Test + public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + // cluster 1 gets scheduled and finishes + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createReplace(newInstantTime, WriteOperationType.CLUSTER); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with cluster 1 + Assertions.assertTrue(candidateInstants.size() == 1); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @Test + public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant); + // replace 1 gets scheduled and finished + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with replace 1 + Assertions.assertTrue(candidateInstants.size() == 1); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + private void createCommit(String instantTime) throws Exception { + String fileId1 = "file-1"; + String fileId2 = "file-2"; + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId("file-1"); + commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); + commitMetadata.setOperationType(WriteOperationType.INSERT); + HoodieTestTable.of(metaClient) + .addCommit(instantTime, commitMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private HoodieCommitMetadata createCommitMetadata(String instantTime) { + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId("file-1"); + commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); + commitMetadata.setOperationType(WriteOperationType.INSERT); + return commitMetadata; + } + + private void createInflightCommit(String instantTime) throws Exception { + String fileId1 = "file-" + instantTime + "-1"; + String fileId2 = "file-" + instantTime + "-2"; + HoodieTestTable.of(metaClient) + .addInflightCommit(instantTime) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private void createCompactionRequested(String instantTime) throws Exception { + String fileId1 = "file-1"; + HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan(); + compactionPlan.setVersion(TimelineLayoutVersion.CURR_VERSION); + HoodieCompactionOperation operation = new HoodieCompactionOperation(); + operation.setFileId(fileId1); + operation.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + operation.setDataFilePath("/file-1"); + operation.setDeltaFilePaths(Arrays.asList("/file-1")); + compactionPlan.setOperations(Arrays.asList(operation)); + HoodieTestTable.of(metaClient) + .addRequestedCompaction(instantTime, compactionPlan); + } + + private void createCompaction(String instantTime) throws Exception { + String fileId1 = "file-1"; + String fileId2 = "file-2"; + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + commitMetadata.setOperationType(WriteOperationType.COMPACT); + commitMetadata.setCompacted(true); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId("file-1"); + commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); + HoodieTestTable.of(metaClient) + .addCommit(instantTime, commitMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private void createReplaceRequested(String instantTime) throws Exception { + String fileId1 = "file-1"; + String fileId2 = "file-2"; + + // create replace instant to mark fileId1 as deleted + HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata(); + requestedReplaceMetadata.setOperationType(WriteOperationType.CLUSTER.name()); + HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan(); + HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup(); + HoodieSliceInfo sliceInfo = new HoodieSliceInfo(); + sliceInfo.setFileId(fileId1); + sliceInfo.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + clusteringGroup.setSlices(Arrays.asList(sliceInfo)); + clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup)); + requestedReplaceMetadata.setClusteringPlan(clusteringPlan); + requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION); + HoodieTestTable.of(metaClient) + .addRequestedReplace(instantTime, requestedReplaceMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private void createReplace(String instantTime, WriteOperationType writeOperationType) throws Exception { + String fileId1 = "file-1"; + String fileId2 = "file-2"; + + // create replace instant to mark fileId1 as deleted + HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata(); + Map> partitionFileIds = new HashMap<>(); + partitionFileIds.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, Arrays.asList(fileId2)); + replaceMetadata.setPartitionToReplaceFileIds(partitionFileIds); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId("file-1"); + replaceMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); + replaceMetadata.setOperationType(writeOperationType); + // create replace instant to mark fileId1 as deleted + HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata(); + requestedReplaceMetadata.setOperationType(WriteOperationType.CLUSTER.name()); + HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan(); + HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup(); + HoodieSliceInfo sliceInfo = new HoodieSliceInfo(); + sliceInfo.setFileId(fileId1); + sliceInfo.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + clusteringGroup.setSlices(Arrays.asList(sliceInfo)); + clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup)); + requestedReplaceMetadata.setClusteringPlan(clusteringPlan); + requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION); + HoodieTestTable.of(metaClient) + .addReplaceCommit(instantTime, requestedReplaceMetadata, replaceMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java new file mode 100644 index 0000000000000..5e36399a65863 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.RetryOneTime; +import org.apache.curator.test.TestingServer; +import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +public class TestZookeeperBasedLockProvider { + + private static final Logger LOG = LogManager.getLogger(TestZookeeperBasedLockProvider.class); + + private static TestingServer server; + private static CuratorFramework client; + private static String basePath = "/hudi/test/lock"; + private static String key = "table1"; + private static LockConfiguration lockConfiguration; + + @BeforeAll + public static void setup() { + while (server == null) { + try { + server = new TestingServer(); + CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder(); + client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1000)).build(); + } catch (Exception e) { + LOG.error("Getting bind exception - retrying to allocate server"); + server = null; + } + } + Properties properties = new Properties(); + properties.setProperty(ZK_BASE_PATH_PROP, basePath); + properties.setProperty(ZK_LOCK_KEY_PROP, key); + properties.setProperty(ZK_CONNECT_URL_PROP, server.getConnectString()); + properties.setProperty(ZK_BASE_PATH_PROP, server.getTempDirectory().getAbsolutePath()); + properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, "10000"); + properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, "10000"); + properties.setProperty(ZK_LOCK_KEY_PROP, "key"); + properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, "1000"); + lockConfiguration = new LockConfiguration(properties); + } + + @Test + public void testAcquireLock() { + ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client); + Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS)); + zookeeperBasedLockProvider.unlock(); + } + + @Test + public void testUnLock() { + ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client); + Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS)); + zookeeperBasedLockProvider.unlock(); + zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS); + } + + @Test + public void testReentrantLock() { + ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client); + Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS)); + try { + zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS); + Assertions.fail(); + } catch (HoodieLockException e) { + // expected + } + zookeeperBasedLockProvider.unlock(); + } + + @Test + public void testUnlockWithoutLock() { + ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client); + zookeeperBasedLockProvider.unlock(); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java new file mode 100644 index 0000000000000..d8a51708f6df5 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java @@ -0,0 +1,213 @@ +/* + * 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.utils; + +import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieInstantInfo; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata; +import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.client.utils.MetadataConversionUtils; +import org.apache.hudi.common.HoodieCleanStat; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +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.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestMetadataConversionUtils extends HoodieCommonTestHarness { + + @BeforeEach + public void init() throws IOException { + initMetaClient(); + } + + @Test + public void testCompletedClean() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createCleanMetadata(newCommitTime); + HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( + new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, newCommitTime), metaClient); + assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); + assertEquals(metaEntry.getHoodieCleanMetadata().getStartCleanTime(), newCommitTime); + } + + @Test + public void testCompletedReplace() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE); + HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( + new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); + assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); + assertEquals(metaEntry.getHoodieReplaceCommitMetadata().getOperationType(), WriteOperationType.INSERT_OVERWRITE.toString()); + } + + @Test + public void testCompletedCommitOrDeltaCommit() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createCommitMetadata(newCommitTime); + HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient); + assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); + assertEquals(metaEntry.getHoodieCommitMetadata().getOperationType(), WriteOperationType.INSERT.toString()); + } + + @Test + public void testCompletedRollback() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createRollbackMetadata(newCommitTime); + HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( + new HoodieInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, newCommitTime), metaClient); + assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); + assertEquals(metaEntry.getHoodieRollbackMetadata().getStartRollbackTime(), newCommitTime); + } + + @Test + public void testCompletedCompaction() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createCompactionMetadata(newCommitTime); + HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient); + assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); + assertEquals(metaEntry.getHoodieCommitMetadata().getOperationType(), WriteOperationType.COMPACT.toString()); + } + + @Test + public void testConvertCommitMetadata() { + HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata(); + hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT); + org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = MetadataConversionUtils + .convertCommitMetadata(hoodieCommitMetadata); + assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString()); + } + + private void createCompactionMetadata(String instantTime) throws Exception { + String fileId1 = "file-" + instantTime + "-1"; + String fileId2 = "file-" + instantTime + "-2"; + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + commitMetadata.setOperationType(WriteOperationType.COMPACT); + commitMetadata.setCompacted(true); + HoodieTestTable.of(metaClient) + .addCommit(instantTime, commitMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private void createRollbackMetadata(String instantTime) throws Exception { + HoodieRollbackMetadata rollbackMetadata = new HoodieRollbackMetadata(); + rollbackMetadata.setCommitsRollback(Arrays.asList(instantTime)); + rollbackMetadata.setStartRollbackTime(instantTime); + HoodieRollbackPartitionMetadata rollbackPartitionMetadata = new HoodieRollbackPartitionMetadata(); + rollbackPartitionMetadata.setPartitionPath("p1"); + rollbackPartitionMetadata.setSuccessDeleteFiles(Arrays.asList("f1")); + rollbackPartitionMetadata.setFailedDeleteFiles(new ArrayList<>()); + rollbackPartitionMetadata.setWrittenLogFiles(new HashMap<>()); + rollbackPartitionMetadata.setRollbackLogFiles(new HashMap<>()); + Map partitionMetadataMap = new HashMap<>(); + partitionMetadataMap.put("p1", rollbackPartitionMetadata); + rollbackMetadata.setPartitionMetadata(partitionMetadataMap); + rollbackMetadata.setInstantsRollback(Arrays.asList(new HoodieInstantInfo("1", HoodieTimeline.COMMIT_ACTION))); + HoodieTestTable.of(metaClient) + .addRollback(instantTime, rollbackMetadata); + } + + private void createCommitMetadata(String instantTime) throws Exception { + String fileId1 = "file-" + instantTime + "-1"; + String fileId2 = "file-" + instantTime + "-2"; + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + commitMetadata.setOperationType(WriteOperationType.INSERT); + HoodieTestTable.of(metaClient) + .addCommit(instantTime, commitMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private void createReplace(String instantTime, WriteOperationType writeOperationType) throws Exception { + String fileId1 = "file-1"; + String fileId2 = "file-2"; + + // create replace instant to mark fileId1 as deleted + HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata(); + Map> partitionFileIds = new HashMap<>(); + partitionFileIds.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, Arrays.asList(fileId2)); + replaceMetadata.setPartitionToReplaceFileIds(partitionFileIds); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId("file-1"); + replaceMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); + replaceMetadata.setOperationType(writeOperationType); + // create replace instant to mark fileId1 as deleted + HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata(); + requestedReplaceMetadata.setOperationType(WriteOperationType.INSERT_OVERWRITE.name()); + HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan(); + HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup(); + HoodieSliceInfo sliceInfo = new HoodieSliceInfo(); + clusteringGroup.setSlices(Arrays.asList(sliceInfo)); + clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup)); + requestedReplaceMetadata.setClusteringPlan(clusteringPlan); + requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION); + HoodieTestTable.of(metaClient) + .addReplaceCommit(instantTime, requestedReplaceMetadata, replaceMetadata) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private void createCleanMetadata(String instantTime) throws IOException { + HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(), + CleanPlanV2MigrationHandler.VERSION, new HashMap<>()); + HoodieCleanStat cleanStats = new HoodieCleanStat( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, + HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEFAULT_PARTITION_PATHS.length)], + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + instantTime); + HoodieCleanMetadata cleanMetadata = convertCleanMetadata(instantTime, Option.of(0L), Collections.singletonList(cleanStats)); + HoodieTestTable.of(metaClient).addClean(instantTime, cleanerPlan, cleanMetadata); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 5d86d0a87451f..f8be0f1c7f06e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -134,7 +135,7 @@ public List upsert(List> records, String instantTim HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.UPSERT); + preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient()); final HoodieRecord record = records.get(0); final boolean isDelta = table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ); final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(record, isDelta, getConfig(), @@ -156,7 +157,7 @@ public List insert(List> records, String instantTim HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT); + preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); // create the write handle if not exists final HoodieRecord record = records.get(0); final boolean isDelta = table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ); @@ -193,7 +194,7 @@ public List bulkInsertPreppedRecords(List> preppedR public List delete(List keys, String instantTime) { HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); - preWrite(instantTime, WriteOperationType.DELETE); + preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient()); HoodieWriteMetadata> result = table.delete(context,instantTime, keys); return postWrite(result, instantTime, table); } @@ -313,6 +314,15 @@ public void cleanHandles() { return writeHandle; } + @Override + protected void preCommit(String instantTime, HoodieCommitMetadata metadata) { + // Create a Hoodie table after startTxn which encapsulated the commits and files visible. + // Important to create this after the lock to ensure latest commits show up in the timeline without need for reload + HoodieTable table = createTable(config, hadoopConf); + TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), + Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner()); + } + private HoodieTable>, List, List> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { if (operationType == WriteOperationType.DELETE) { setWriteSchemaForDeletes(metaClient); 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 e7f1d592e5bf0..167b3766e51aa 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 @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -43,6 +44,7 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; 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.FlinkInsertPreppedCommitActionExecutor; @@ -265,6 +267,17 @@ public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { throw new HoodieNotSupportedException("Bootstrap is not supported yet"); } + /** + * @param context HoodieEngineContext + * @param instantTime Instant Time for scheduling cleaning + * @param extraMetadata additional metadata to write into plan + * @return + */ + @Override + public Option scheduleCleaning(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + return new FlinkScheduleCleanActionExecutor(context, config, this, instantTime, extraMetadata).execute(); + } + @Override public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) { return new FlinkCleanActionExecutor(context, config, this, cleanInstantTime).execute(); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkScheduleCleanActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkScheduleCleanActionExecutor.java new file mode 100644 index 0000000000000..75da54e31cf9d --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkScheduleCleanActionExecutor.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.clean; + +import java.util.List; +import java.util.Map; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +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.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +public class FlinkScheduleCleanActionExecutor extends + BaseCleanPlanActionExecutor>, List, List> { + + private static final Logger LOG = LogManager.getLogger(FlinkScheduleCleanActionExecutor.class); + + public FlinkScheduleCleanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected Option createCleanerPlan() { + return super.execute(); + } +} 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 index e0bbc25efd52e..5df46a65bc5ee 100644 --- 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 @@ -143,7 +143,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - + syncTableMetadata(); try { LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index f252e8b94d743..8b7cb198bc9ec 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -97,7 +97,7 @@ public List upsert(List> records, HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.UPSERT); + preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient()); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -111,7 +111,7 @@ public List upsertPreppedRecords(List> preppedRecor HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.UPSERT_PREPPED); + preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient()); HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -121,7 +121,7 @@ public List insert(List> records, String instantTim HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT); + preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); HoodieWriteMetadata> result = table.insert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -135,7 +135,7 @@ public List insertPreppedRecords(List> preppedRecor HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT_PREPPED); + preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient()); HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -165,7 +165,7 @@ public List delete(List keys, String instantTime) { HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); - preWrite(instantTime, WriteOperationType.DELETE); + preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient()); HoodieWriteMetadata> result = table.delete(context,instantTime, keys); return postWrite(result, instantTime, table); } 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 157e11a55d6de..06e66a1a02f91 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 @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -38,6 +39,7 @@ 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; @@ -175,6 +177,11 @@ public void rollbackBootstrap(HoodieEngineContext context, throw new HoodieNotSupportedException("RollbackBootstrap is not supported yet"); } + @Override + public Option scheduleCleaning(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + return new JavaScheduleCleanActionExecutor<>(context, config, this, instantTime, extraMetadata).execute(); + } + @Override public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaScheduleCleanActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaScheduleCleanActionExecutor.java new file mode 100644 index 0000000000000..05d19a63ef160 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaScheduleCleanActionExecutor.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.clean; + +import java.util.List; +import java.util.Map; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +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.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +public class JavaScheduleCleanActionExecutor extends + BaseCleanPlanActionExecutor>, List, List> { + + private static final Logger LOG = LogManager.getLogger(JavaScheduleCleanActionExecutor.class); + + public JavaScheduleCleanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected Option createCleanerPlan() { + return super.execute(); + } +} 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/BaseJavaCommitActionExecutor.java index a4a6a4f92108c..a2ea1c9c0a837 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/BaseJavaCommitActionExecutor.java @@ -200,7 +200,6 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - try { LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); 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 17b174279b4d3..af3f3c2bca6da 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 @@ -430,11 +430,14 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { WriteStatus writeStatus = ws.get(0).get(0); String fileId = writeStatus.getFileId(); metaClient.getFs().create(new Path(Paths.get(basePath, ".hoodie", "000.commit").toString())).close(); + //TODO : Find race condition that causes the timeline sometime to reflect 000.commit and sometimes not + final HoodieJavaCopyOnWriteTable reloadedTable = (HoodieJavaCopyOnWriteTable) HoodieJavaTable.create(config, context, HoodieTableMetaClient.reload(metaClient)); + final List updates = dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts); String partitionPath = writeStatus.getPartitionPath(); long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); - BaseJavaCommitActionExecutor newActionExecutor = new JavaUpsertCommitActionExecutor(context, config, table, + BaseJavaCommitActionExecutor newActionExecutor = new JavaUpsertCommitActionExecutor(context, config, reloadedTable, instantTime, updates); taskContextSupplier.reset(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 091f1dafa8051..8a0eee59a5486 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -22,15 +22,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; 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.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -52,6 +55,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.compact.SparkCompactHelpers; +import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade; import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -146,7 +150,7 @@ public JavaRDD upsert(JavaRDD> records, String inst HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.UPSERT); + preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient()); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -159,7 +163,7 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); table.validateUpsertSchema(); - preWrite(instantTime, WriteOperationType.UPSERT_PREPPED); + preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient()); HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -169,7 +173,7 @@ public JavaRDD insert(JavaRDD> records, String inst HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT); + preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); HoodieWriteMetadata> result = table.insert(context,instantTime, records); return postWrite(result, instantTime, table); } @@ -179,7 +183,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT_PREPPED); + preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient()); HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -194,7 +198,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE); + preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient()); HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records); return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); } @@ -210,7 +214,7 @@ public HoodieWriteResult insertOverwrite(JavaRDD> records, final public HoodieWriteResult insertOverwriteTable(JavaRDD> records, final String instantTime) { HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE); + preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient()); HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records); return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); } @@ -225,7 +229,7 @@ public JavaRDD bulkInsert(JavaRDD> records, String HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.BULK_INSERT); + preWrite(instantTime, WriteOperationType.BULK_INSERT, table.getMetaClient()); HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner); return postWrite(result, instantTime, table); } @@ -235,7 +239,7 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime); table.validateInsertSchema(); - preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED); + preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient()); HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner); return postWrite(result, instantTime, table); } @@ -243,14 +247,14 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr @Override public JavaRDD delete(JavaRDD keys, String instantTime) { HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); - preWrite(instantTime, WriteOperationType.DELETE); + preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient()); HoodieWriteMetadata> result = table.delete(context,instantTime, keys); return postWrite(result, instantTime, table); } public HoodieWriteResult deletePartitions(List partitions, String instantTime) { HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE_PARTITION, instantTime); - preWrite(instantTime, WriteOperationType.DELETE_PARTITION); + preWrite(instantTime, WriteOperationType.DELETE_PARTITION, table.getMetaClient()); HoodieWriteMetadata> result = table.deletePartitions(context,instantTime, partitions); return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); } @@ -311,6 +315,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD compact(String compactionInstantTime, boolean shouldComplete) { HoodieSparkTable table = HoodieSparkTable.create(config, context); + preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient()); HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { @@ -321,7 +326,7 @@ protected JavaRDD compact(String compactionInstantTime, boolean sho HoodieWriteMetadata> compactionMetadata = table.compact(context, compactionInstantTime); JavaRDD statuses = compactionMetadata.getWriteStatuses(); if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) { - completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime); + completeTableService(TableServiceType.COMPACT, compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime); } return statuses; } @@ -329,6 +334,7 @@ protected JavaRDD compact(String compactionInstantTime, boolean sho @Override public HoodieWriteMetadata> cluster(String clusteringInstant, boolean shouldComplete) { HoodieSparkTable table = HoodieSparkTable.create(config, context); + preWrite(clusteringInstant, WriteOperationType.CLUSTER, table.getMetaClient()); HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline(); HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant); if (pendingClusteringTimeline.containsInstant(inflightInstant)) { @@ -339,13 +345,14 @@ public HoodieWriteMetadata> cluster(String clusteringInstan LOG.info("Starting clustering at " + clusteringInstant); HoodieWriteMetadata> clusteringMetadata = table.cluster(context, clusteringInstant); JavaRDD statuses = clusteringMetadata.getWriteStatuses(); + // TODO : Where is shouldComplete used ? if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) { - completeClustering((HoodieReplaceCommitMetadata) clusteringMetadata.getCommitMetadata().get(), statuses, table, clusteringInstant); + completeTableService(TableServiceType.CLUSTER, clusteringMetadata.getCommitMetadata().get(), statuses, table, clusteringInstant); } return clusteringMetadata; } - protected void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD writeStatuses, + private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD writeStatuses, HoodieTable>, JavaRDD, JavaRDD> table, String clusteringCommitTime) { @@ -380,11 +387,44 @@ protected void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD< @Override protected HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); - new SparkUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); - return getTableAndInitCtx(metaClient, operationType); + AbstractUpgradeDowngrade upgradeDowngrade = new SparkUpgradeDowngrade(metaClient, config, context); + if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) { + if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { + this.txnManager.beginTransaction(); + try { + // Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits + this.rollbackFailedWrites(getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER)); + new SparkUpgradeDowngrade(metaClient, config, context) + .run(metaClient, HoodieTableVersion.current(), config, context, instantTime); + } finally { + this.txnManager.endTransaction(); + } + } else { + upgradeDowngrade.run(metaClient, HoodieTableVersion.current(), config, context, instantTime); + } + } + return getTableAndInitCtx(metaClient, operationType, instantTime); } - private HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { + // TODO : To enforce priority between table service and ingestion writer, use transactions here and invoke strategy + private void completeTableService(TableServiceType tableServiceType, HoodieCommitMetadata metadata, JavaRDD writeStatuses, + HoodieTable>, JavaRDD, JavaRDD> table, + String commitInstant) { + + switch (tableServiceType) { + case CLUSTER: + completeClustering((HoodieReplaceCommitMetadata) metadata, writeStatuses, table, commitInstant); + break; + case COMPACT: + completeCompaction(metadata, writeStatuses, table, commitInstant); + break; + default: + throw new IllegalArgumentException("This table service is not valid " + tableServiceType); + } + } + + private HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx( + HoodieTableMetaClient metaClient, WriteOperationType operationType, String instantTime) { if (operationType == WriteOperationType.DELETE) { setWriteSchemaForDeletes(metaClient); } @@ -408,6 +448,15 @@ public void syncTableMetadata() { } } + @Override + protected void preCommit(String instantTime, HoodieCommitMetadata metadata) { + // Create a Hoodie table after startTxn which encapsulated the commits and files visible. + // Important to create this after the lock to ensure latest commits show up in the timeline without need for reload + HoodieTable table = createTable(config, hadoopConf); + TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), + Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner()); + } + @Override protected void initWrapperFSMetrics() { if (config.isMetricsOn()) { 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 12867e636cf03..98063f6cf0ccd 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 @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -45,8 +46,9 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; import org.apache.hudi.table.action.clean.SparkCleanActionExecutor; -import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor; +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; @@ -176,6 +178,11 @@ public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); } + @Override + public Option scheduleCleaning(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + return new SparkCleanPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute(); + } + public Iterator> handleUpdate(String instantTime, String partitionPath, String fileId, Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException { // these are updates 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/SparkBootstrapCommitActionExecutor.java index 1aca28b6d8f79..4f204eabd1830 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/SparkBootstrapCommitActionExecutor.java @@ -57,9 +57,12 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +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.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; @@ -222,6 +225,17 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta 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); @@ -237,7 +251,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta // Finalize write finalizeWrite(instantTime, stats, result); - + syncTableMetadata(); // add in extra metadata if (extraMetadata.isPresent()) { extraMetadata.get().forEach(metadata::addMetadata); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanPlanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanPlanActionExecutor.java new file mode 100644 index 0000000000000..f5529a8b740a1 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanPlanActionExecutor.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.clean; + +import org.apache.hudi.avro.model.HoodieCleanerPlan; +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.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkCleanPlanActionExecutor extends + BaseCleanPlanActionExecutor>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkCleanPlanActionExecutor.class); + + public SparkCleanPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected Option createCleanerPlan() { + return super.execute(); + } + +} 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/BaseSparkCommitActionExecutor.java index a99b00162d68f..aa199c2b5008e 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/BaseSparkCommitActionExecutor.java @@ -38,12 +38,15 @@ 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.HoodieMetadataException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.SparkLazyInsertIterable; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieSortedMergeHandle; import org.apache.hudi.io.storage.HoodieConcatHandle; +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.WorkloadProfile; @@ -244,7 +247,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - + syncTableMetadata(); try { LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); @@ -340,6 +343,17 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, partitionPath, fileId, dataFileToBeMerged, taskContextSupplier); } + @Override + public 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); + } + } + @Override public Iterator> handleInsert(String idPfx, Iterator> recordItr) throws Exception { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java new file mode 100644 index 0000000000000..a56789bd1bff9 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client; + +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.TableServiceType; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieWriteConflictException; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieClientMultiWriter extends HoodieClientTestBase { + + public void setUpMORTestTable() throws IOException { + cleanupResources(); + initPath(); + initSparkContexts(); + initTestDataGenerator(); + initFileSystem(); + fs.mkdirs(new Path(basePath)); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, HoodieFileFormat.PARQUET); + initTestDataGenerator(); + } + + @AfterEach + public void clean() throws IOException { + cleanupResources(); + } + + @ParameterizedTest + @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) + public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws Exception { + if (tableType == HoodieTableType.MERGE_ON_READ) { + setUpMORTestTable(); + } + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP, basePath + "/.hoodie/.locks"); + HoodieWriteConfig cfg = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class) + .build()).withAutoCommit(false).withProperties(properties).build(); + // Create the first commit + createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200); + try { + ExecutorService executors = Executors.newFixedThreadPool(2); + SparkRDDWriteClient client1 = getHoodieWriteClient(cfg); + SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); + Future future1 = executors.submit(() -> { + String newCommitTime = "004"; + int numRecords = 100; + String commitTimeBetweenPrevAndNew = "002"; + try { + createCommitWithUpserts(cfg, client1, "002", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); + } catch (Exception e1) { + Assertions.assertTrue(e1 instanceof HoodieWriteConflictException); + throw new RuntimeException(e1); + } + }); + Future future2 = executors.submit(() -> { + String newCommitTime = "005"; + int numRecords = 100; + String commitTimeBetweenPrevAndNew = "002"; + try { + createCommitWithUpserts(cfg, client2, "002", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); + } catch (Exception e2) { + Assertions.assertTrue(e2 instanceof HoodieWriteConflictException); + throw new RuntimeException(e2); + } + }); + future1.get(); + future2.get(); + Assertions.fail("Should not reach here, this means concurrent writes were handled incorrectly"); + } catch (Exception e) { + // Expected to fail due to overlapping commits + } + } + + @ParameterizedTest + @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) + public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception { + // create inserts X 1 + if (tableType == HoodieTableType.MERGE_ON_READ) { + setUpMORTestTable(); + } + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP, basePath + "/.hoodie/.locks"); + // Disabling embedded timeline server, it doesn't work with multiwriter + HoodieWriteConfig cfg = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false) + .withInlineCompaction(false).withAsyncClean(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withMaxNumDeltaCommitsBeforeCompaction(2).build()) + .withEmbeddedTimelineServerEnabled(false) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType( + FileSystemViewStorageType.MEMORY).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClusteringNumCommits(1).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class) + .build()).withAutoCommit(false).withProperties(properties).build(); + Set validInstants = new HashSet<>(); + // Create the first commit with inserts + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + createCommitWithInserts(cfg, client, "000", "001", 200); + validInstants.add("001"); + // Create 2 commits with upserts + createCommitWithUpserts(cfg, client, "001", "000", "002", 100); + createCommitWithUpserts(cfg, client, "002", "000", "003", 100); + validInstants.add("002"); + validInstants.add("003"); + ExecutorService executors = Executors.newFixedThreadPool(2); + SparkRDDWriteClient client1 = getHoodieWriteClient(cfg); + SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); + // Create upserts, schedule cleaning, schedule compaction in parallel + Future future1 = executors.submit(() -> { + String newCommitTime = "004"; + int numRecords = 100; + String commitTimeBetweenPrevAndNew = "002"; + try { + createCommitWithUpserts(cfg, client1, "003", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); + if (tableType == HoodieTableType.MERGE_ON_READ) { + Assertions.fail("Conflicts not handled correctly"); + } + validInstants.add("004"); + } catch (Exception e1) { + if (tableType == HoodieTableType.MERGE_ON_READ) { + Assertions.assertTrue(e1 instanceof HoodieWriteConflictException); + } + } + }); + Future future2 = executors.submit(() -> { + try { + client2.scheduleTableService("005", Option.empty(), TableServiceType.COMPACT); + } catch (Exception e2) { + if (tableType == HoodieTableType.MERGE_ON_READ) { + throw new RuntimeException(e2); + } + } + }); + Future future3 = executors.submit(() -> { + try { + client2.scheduleTableService("006", Option.empty(), TableServiceType.CLEAN); + } catch (Exception e2) { + throw new RuntimeException(e2); + } + }); + future1.get(); + future2.get(); + future3.get(); + // Create inserts, run cleaning, run compaction in parallel + future1 = executors.submit(() -> { + String newCommitTime = "007"; + int numRecords = 100; + try { + createCommitWithInserts(cfg, client1, "003", newCommitTime, numRecords); + validInstants.add("007"); + } catch (Exception e1) { + throw new RuntimeException(e1); + } + }); + future2 = executors.submit(() -> { + try { + JavaRDD writeStatusJavaRDD = (JavaRDD) client2.compact("005"); + client2.commitCompaction("005", writeStatusJavaRDD, Option.empty()); + validInstants.add("005"); + } catch (Exception e2) { + if (tableType == HoodieTableType.MERGE_ON_READ) { + throw new RuntimeException(e2); + } + } + }); + future3 = executors.submit(() -> { + try { + client2.clean("006", false); + validInstants.add("006"); + } catch (Exception e2) { + throw new RuntimeException(e2); + } + }); + future1.get(); + future2.get(); + future3.get(); + Set completedInstants = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp) + .collect(Collectors.toSet()); + Assertions.assertTrue(validInstants.containsAll(completedInstants)); + } + + @ParameterizedTest + @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) + public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) throws Exception { + if (tableType == HoodieTableType.MERGE_ON_READ) { + setUpMORTestTable(); + } + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP, basePath + "/.hoodie/.locks"); + HoodieWriteConfig cfg = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoClean(false).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClusteringNumCommits(1).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class) + .build()).withAutoCommit(false).withProperties(properties).build(); + // Create the first commit + createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200); + // Start another inflight commit + String newCommitTime = "003"; + int numRecords = 100; + SparkRDDWriteClient client1 = getHoodieWriteClient(cfg); + String commitTimeBetweenPrevAndNew = "002"; + JavaRDD result1 = updateBatch(cfg, client1, newCommitTime, "001", + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, + numRecords, 200, 2); + // Start and finish another commit while the previous writer for commit 003 is running + newCommitTime = "004"; + SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); + JavaRDD result2 = updateBatch(cfg, client2, newCommitTime, "001", + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, + numRecords, 200, 2); + client2.commit(newCommitTime, result2); + // Schedule and run clustering while previous writer for commit 003 is running + SparkRDDWriteClient client3 = getHoodieWriteClient(cfg); + // schedule clustering + Option clusterInstant = client3.scheduleTableService(Option.empty(), TableServiceType.CLUSTER); + client3.cluster(clusterInstant.get(), true); + // Attempt to commit the inflight commit 003 + try { + client1.commit("003", result1); + Assertions.fail("Should have thrown a concurrent conflict exception"); + } catch (Exception e) { + // Expected + } + } + + private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, + String prevCommitTime, String newCommitTime, int numRecords) throws Exception { + // Finish first base commmit + JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::bulkInsert, + false, false, numRecords); + assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); + } + + private void createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommit, + String commitTimeBetweenPrevAndNew, String newCommitTime, int numRecords) + throws Exception { + JavaRDD result = updateBatch(cfg, client, newCommitTime, prevCommit, + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, + numRecords, 200, 2); + client.commit(newCommitTime, result); + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index 41ee4b2085b86..5c2b6152112cf 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -34,7 +35,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; -import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -66,8 +66,8 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -97,6 +97,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutorService; @@ -104,7 +105,6 @@ import java.util.concurrent.Future; import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.Properties; import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0; import static org.apache.hudi.common.testutils.FileCreateUtils.getBaseFileCountsForPaths; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java index 042013d3af3e5..5b1888101aac9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java @@ -22,6 +22,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; @@ -487,7 +488,8 @@ public void testConvertCommitMetadata() { HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata); + org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = MetadataConversionUtils + .convertCommitMetadata(hoodieCommitMetadata); assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java index 5809ab210a6c9..b8e54f05a354b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -557,10 +557,6 @@ public void testSync(HoodieTableType tableType) throws Exception { try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { // Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details client.syncTableMetadata(); - } - - // Enable metadata table and ensure it is synced - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { client.restoreToInstant(restoreToInstant); assertFalse(metadata(client).isInSync()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index fd578bd8b70bc..83bb684294024 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -594,13 +594,8 @@ private List runCleaner(HoodieWriteConfig config, boolean simul }); }); metaClient.reloadActiveTimeline().revertToInflight(completedCleanInstant); - HoodieCleanMetadata newCleanMetadata = writeClient.clean(makeNewCommitTime(firstCommitSequence + 1)); - // No new clean metadata would be created. Only the previous one will be retried - assertNull(newCleanMetadata); - HoodieCleanMetadata cleanMetadata2 = CleanerUtils.getCleanerMetadata(metaClient, completedCleanInstant); - assertEquals(cleanMetadata1.getEarliestCommitToRetain(), cleanMetadata2.getEarliestCommitToRetain()); - assertEquals(cleanMetadata1.getTotalFilesDeleted(), cleanMetadata2.getTotalFilesDeleted()); - assertEquals(cleanMetadata1.getPartitionMetadata().keySet(), cleanMetadata2.getPartitionMetadata().keySet()); + // retry clean operation again + writeClient.clean(); final HoodieCleanMetadata retriedCleanMetadata = CleanerUtils.getCleanerMetadata(HoodieTableMetaClient.reload(metaClient), completedCleanInstant); cleanMetadata1.getPartitionMetadata().keySet().forEach(k -> { HoodieCleanPartitionMetadata p1 = cleanMetadata1.getPartitionMetadata().get(k); 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 c054bc4602f85..3e03c05366284 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 @@ -408,7 +408,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { .withStorageConfig(HoodieStorageConfig.newBuilder() .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build(); metaClient = HoodieTableMetaClient.reload(metaClient); - final HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, 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); @@ -425,6 +425,7 @@ 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)); final List> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 72b277128162d..951e17ca36c53 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -17,6 +17,13 @@ package org.apache.hudi.testutils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.SparkTaskContextSupplier; @@ -34,14 +41,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hudi.table.WorkloadStat; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java b/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java new file mode 100644 index 0000000000000..d9be6a6475b01 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java @@ -0,0 +1,69 @@ +/* + * 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.common.config; + +import java.io.Serializable; +import java.util.Properties; + +/** + * Configuration for managing locks. Since this configuration needs to be shared with HiveMetaStore based lock, + * which is in a different package than other lock providers, we use this as a data transfer object in hoodie-common + */ +public class LockConfiguration implements Serializable { + + public static final String LOCK_PREFIX = "hoodie.writer.lock."; + public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "wait_time_ms_between_retry"; + public static final String DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L); + public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "client.wait_time_ms_between_retry"; + public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(10000L); + public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP = LOCK_PREFIX + "num_retries"; + public static final String DEFAULT_LOCK_ACQUIRE_NUM_RETRIES = String.valueOf(3); + public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = LOCK_PREFIX + "client.num_retries"; + public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES = String.valueOf(0); + public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = LOCK_PREFIX + "wait_time_ms"; + public static final int DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS = 60 * 1000; + // configs for file system based locks. NOTE: This only works for DFS with atomic create/delete operation + public static final String FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "filesystem."; + public static final String FILESYSTEM_LOCK_PATH_PROP = FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX + "path"; + // configs for metastore based locks + public static final String HIVE_METASTORE_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "hivemetastore."; + public static final String HIVE_DATABASE_NAME_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "database"; + public static final String HIVE_TABLE_NAME_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "table"; + // Zookeeper configs for zk based locks + public static final String ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "zookeeper."; + public static final String ZK_BASE_PATH_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "zk_base_path"; + public static final String ZK_SESSION_TIMEOUT_MS_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "zk_session_timeout_ms"; + public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000; + public static final String ZK_CONNECTION_TIMEOUT_MS_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "zk_connection_timeout_ms"; + public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000; + public static final String ZK_CONNECT_URL_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "url"; + public static final String ZK_PORT_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "port"; + public static final String ZK_LOCK_KEY_PROP = LOCK_PREFIX + "lock_key"; + + private final TypedProperties props; + + public LockConfiguration(Properties props) { + this.props = new TypedProperties(props); + } + + public TypedProperties getConfig() { + return props; + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java b/hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java new file mode 100644 index 0000000000000..7d8e527384542 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java @@ -0,0 +1,56 @@ +/* + * 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.common.lock; + +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; + +/** + * Pluggable lock implementations using this provider class. + */ +public interface LockProvider extends Lock, AutoCloseable { + + @Override + default void lockInterruptibly() { + throw new UnsupportedOperationException(); + } + + @Override + default void lock() { + throw new UnsupportedOperationException(); + } + + @Override + default boolean tryLock() { + throw new UnsupportedOperationException(); + } + + @Override + default Condition newCondition() { + throw new UnsupportedOperationException(); + } + + default T getLock() { + throw new IllegalArgumentException(); + } + + @Override + default void close() { + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/lock/LockState.java b/hudi-common/src/main/java/org/apache/hudi/common/lock/LockState.java new file mode 100644 index 0000000000000..e40d5e5a9dd46 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/lock/LockState.java @@ -0,0 +1,27 @@ +/* + * 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.common.lock; + +/** + * Enum to signal the state of the lock. + */ +public enum LockState { + ACQUIRING, ACQUIRED, ALREADY_ACQUIRED, RELEASING, RELEASED, ALREADY_RELEASED, + FAILED_TO_ACQUIRE, FAILED_TO_RELEASE +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMetadataWrapper.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMetadataWrapper.java new file mode 100644 index 0000000000000..ecc18a7dfaddd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMetadataWrapper.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.common.model; + +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; + +public class HoodieMetadataWrapper { + + private HoodieArchivedMetaEntry avroMetadataFromTimeline; + private HoodieCommitMetadata commitMetadata; + private boolean isAvroMetadata = false; + + public HoodieMetadataWrapper(HoodieArchivedMetaEntry avroMetadataFromTimeline) { + this.avroMetadataFromTimeline = avroMetadataFromTimeline; + this.isAvroMetadata = true; + } + + public HoodieMetadataWrapper(HoodieCommitMetadata commitMetadata) { + this.commitMetadata = commitMetadata; + } + + public HoodieArchivedMetaEntry getMetadataFromTimeline() { + return avroMetadataFromTimeline; + } + + public HoodieCommitMetadata getCommitMetadata() { + return commitMetadata; + } + + public boolean isAvroMetadata() { + return isAvroMetadata; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java new file mode 100644 index 0000000000000..90444a3d61aa2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java @@ -0,0 +1,41 @@ +/* + * 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.common.model; + +import org.apache.hudi.common.table.timeline.HoodieTimeline; + +/** + * Supported runtime table services. + */ +public enum TableServiceType { + COMPACT, CLUSTER, CLEAN; + + public String getAction() { + switch (this) { + case COMPACT: + return HoodieTimeline.COMPACTION_ACTION; + case CLEAN: + return HoodieTimeline.CLEAN_ACTION; + case CLUSTER: + return HoodieTimeline.REPLACE_COMMIT_ACTION; + default: + throw new IllegalArgumentException("Unknown table service " + this); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteConcurrencyMode.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteConcurrencyMode.java new file mode 100644 index 0000000000000..9fe66ca0a0b68 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteConcurrencyMode.java @@ -0,0 +1,66 @@ +/* + * 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.common.model; + +import org.apache.hudi.exception.HoodieException; + +import java.util.Locale; + +/** + * Different concurrency modes for write operations. + */ +public enum WriteConcurrencyMode { + // Only a single writer can perform write ops + SINGLE_WRITER("single_writer"), + // Multiple writer can perform write ops with lazy conflict resolution using locks + OPTIMISTIC_CONCURRENCY_CONTROL("optimistic_concurrency_control"); + + private final String value; + + WriteConcurrencyMode(String value) { + this.value = value; + } + + /** + * Getter for write concurrency mode. + * @return + */ + public String value() { + return value; + } + + /** + * Convert string value to WriteConcurrencyMode. + */ + public static WriteConcurrencyMode fromValue(String value) { + switch (value.toLowerCase(Locale.ROOT)) { + case "single_writer": + return SINGLE_WRITER; + case "optimistic_concurrency_control": + return OPTIMISTIC_CONCURRENCY_CONTROL; + default: + throw new HoodieException("Invalid value of Type."); + } + } + + public boolean supportsOptimisticConcurrencyControl() { + return this == OPTIMISTIC_CONCURRENCY_CONTROL; + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index f237156360847..41cfc8abc20b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -46,6 +46,8 @@ public enum WriteOperationType { DELETE_PARTITION("delete_partition"), // insert overwrite with dynamic partitioning INSERT_OVERWRITE_TABLE("insert_overwrite_table"), + // compact + COMPACT("compact"), // used for old version UNKNOWN("unknown"); @@ -82,6 +84,10 @@ public static WriteOperationType fromValue(String value) { return INSERT_OVERWRITE_TABLE; case "cluster": return CLUSTER; + case "compact": + return COMPACT; + case "unknown": + return UNKNOWN; default: throw new HoodieException("Invalid value of Type."); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index fa494931a3c4b..c7d61a732bb3a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -143,6 +143,12 @@ public HoodieDefaultTimeline findInstantsAfter(String instantTime, int numCommit details); } + @Override + public HoodieTimeline findInstantsAfter(String instantTime) { + return new HoodieDefaultTimeline(instants.stream() + .filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)), details); + } + @Override public HoodieDefaultTimeline findInstantsAfterOrEquals(String commitTime, int numCommits) { return new HoodieDefaultTimeline(instants.stream() diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index b48543c3dfc14..1e366147a6047 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -171,6 +171,11 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline findInstantsAfter(String instantTime, int numCommits); + /** + * Create a new Timeline with all the instants after startTs. + */ + HoodieTimeline findInstantsAfter(String instantTime); + /** * Create a new Timeline with all instants before specified time. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index 9c53cec4e6b48..1bf97c68eb69a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -69,26 +69,43 @@ public static Stream> getAllPendingClu .filter(Option::isPresent).map(Option::get); } + /** + * Get requested replace metadata from timeline. + * @param metaClient + * @param pendingReplaceInstant + * @return + * @throws IOException + */ + public static Option getRequestedReplaceMetadata(HoodieTableMetaClient metaClient, HoodieInstant pendingReplaceInstant) throws IOException { + final HoodieInstant requestedInstant; + if (!pendingReplaceInstant.isRequested()) { + // inflight replacecommit files don't have clustering plan. + // This is because replacecommit inflight can have workload profile for 'insert_overwrite'. + // Get the plan from corresponding requested instant. + requestedInstant = HoodieTimeline.getReplaceCommitRequestedInstant(pendingReplaceInstant.getTimestamp()); + } else { + requestedInstant = pendingReplaceInstant; + } + Option content = metaClient.getActiveTimeline().getInstantDetails(requestedInstant); + if (!content.isPresent() || content.get().length == 0) { + // few operations create requested file without any content. Assume these are not clustering + LOG.warn("No content found in requested file for instant " + pendingReplaceInstant); + return Option.empty(); + } + return Option.of(TimelineMetadataUtils.deserializeRequestedReplaceMetadata(content.get())); + } + + /** + * Get Clustering plan from timeline. + * @param metaClient + * @param pendingReplaceInstant + * @return + */ public static Option> getClusteringPlan(HoodieTableMetaClient metaClient, HoodieInstant pendingReplaceInstant) { try { - final HoodieInstant requestedInstant; - if (!pendingReplaceInstant.isRequested()) { - // inflight replacecommit files don't have clustering plan. - // This is because replacecommit inflight can have workload profile for 'insert_overwrite'. - // Get the plan from corresponding requested instant. - requestedInstant = HoodieTimeline.getReplaceCommitRequestedInstant(pendingReplaceInstant.getTimestamp()); - } else { - requestedInstant = pendingReplaceInstant; - } - Option content = metaClient.getActiveTimeline().getInstantDetails(requestedInstant); - if (!content.isPresent() || content.get().length == 0) { - // few operations create requested file without any content. Assume these are not clustering - LOG.warn("No content found in requested file for instant " + pendingReplaceInstant); - return Option.empty(); - } - HoodieRequestedReplaceMetadata requestedReplaceMetadata = TimelineMetadataUtils.deserializeRequestedReplaceMetadata(content.get()); - if (WriteOperationType.CLUSTER.name().equals(requestedReplaceMetadata.getOperationType())) { - return Option.of(Pair.of(pendingReplaceInstant, requestedReplaceMetadata.getClusteringPlan())); + Option requestedReplaceMetadata = getRequestedReplaceMetadata(metaClient, pendingReplaceInstant); + if (requestedReplaceMetadata.isPresent() && WriteOperationType.CLUSTER.name().equals(requestedReplaceMetadata.get().getOperationType())) { + return Option.of(Pair.of(pendingReplaceInstant, requestedReplaceMetadata.get().getClusteringPlan())); } return Option.empty(); } catch (IOException e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java index 7b4c7c5cad1f2..179055f76226b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java @@ -29,6 +29,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -93,4 +94,28 @@ private static HoodieCommitMetadata buildMetadataFromStats(List + "numReplaceFileIds:" + partitionToReplaceFileIds.values().stream().mapToInt(e -> e.size()).sum()); return commitMetadata; } + + public static HashMap getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(Map> + partitionToWriteStats) { + HashMap fileIdToPath = new HashMap<>(); + // list all partitions paths + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (org.apache.hudi.avro.model.HoodieWriteStat stat : entry.getValue()) { + fileIdToPath.put(stat.getFileId(), stat.getPath()); + } + } + return fileIdToPath; + } + + public static HashMap getFileIdWithoutSuffixAndRelativePaths(Map> + partitionToWriteStats) { + HashMap fileIdToPath = new HashMap<>(); + // list all partitions paths + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + for (HoodieWriteStat stat : entry.getValue()) { + fileIdToPath.put(stat.getFileId(), stat.getPath()); + } + } + return fileIdToPath; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieLockException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieLockException.java new file mode 100644 index 0000000000000..cd1ff7f0eaf02 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieLockException.java @@ -0,0 +1,43 @@ +/* + * 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.exception; + +/** + *

+ * Exception thrown for Hoodie failures. The root of the exception hierarchy. + *

+ *

+ * Hoodie Write clients will throw this exception if unable to acquire a lock. This is a runtime (unchecked) + * exception. + *

+ */ +public class HoodieLockException extends HoodieException { + + public HoodieLockException(String msg) { + super(msg); + } + + public HoodieLockException(Throwable e) { + super(e); + } + + public HoodieLockException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java new file mode 100644 index 0000000000000..f0f6dcbf0ab17 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java @@ -0,0 +1,43 @@ +/* + * 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.exception; + +/** + *

+ * Exception thrown for Hoodie failures. The root of the exception hierarchy. + *

+ *

+ * Hoodie Write clients will throw this exception if unable to commit due to conflicts. This is a runtime (unchecked) + * exception. + *

+ */ +public class HoodieWriteConflictException extends HoodieException { + + public HoodieWriteConflictException(String msg) { + super(msg); + } + + public HoodieWriteConflictException(Throwable e) { + super(e); + } + + public HoodieWriteConflictException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 5f03ef015faf0..5474b184bad54 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -22,14 +22,15 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.exception.HoodieException; @@ -47,6 +48,8 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanerPlan; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRequestedReplaceMetadata; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata; public class FileCreateUtils { @@ -117,6 +120,10 @@ public static void createCommit(String basePath, String instantTime) throws IOEx createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } + public static void createCommit(String basePath, String instantTime, HoodieCommitMetadata metadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + } + public static void createCommit(String basePath, String instantTime, FileSystem fs) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION, fs); } @@ -149,9 +156,8 @@ public static void createReplaceCommit(String basePath, String instantTime, Hood createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); } - public static void createRequestedReplaceCommit(String basePath, String instantTime, HoodieRequestedReplaceMetadata requestedMetadata) throws IOException { - createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION, - TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedMetadata).get()); + public static void createRequestedReplaceCommit(String basePath, String instantTime, HoodieRequestedReplaceMetadata requestedReplaceMetadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION, serializeRequestedReplaceMetadata(requestedReplaceMetadata).get()); } public static void createInflightReplaceCommit(String basePath, String instantTime) throws IOException { @@ -170,6 +176,14 @@ public static void createInflightCleanFile(String basePath, String instantTime, createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION, serializeCleanerPlan(cleanerPlan).get()); } + public static void createInflightRollbackFile(String basePath, String instantTime) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); + } + + public static void createRollbackFile(String basePath, String instantTime, HoodieRollbackMetadata rollbackMetadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION, serializeRollbackMetadata(rollbackMetadata).get()); + } + private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); Files.createDirectories(parentPath); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 46ef14f2077a0..182a8d409ab24 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -23,7 +23,9 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.IOType; @@ -65,6 +67,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightReplaceCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCleanFile; @@ -72,6 +75,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; public class HoodieTestTable { @@ -155,6 +159,15 @@ public HoodieTestTable addCommit(String instantTime) throws Exception { return this; } + public HoodieTestTable addCommit(String instantTime, HoodieCommitMetadata metadata) throws Exception { + createRequestedCommit(basePath, instantTime); + createInflightCommit(basePath, instantTime); + createCommit(basePath, instantTime, metadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addDeltaCommit(String instantTime) throws Exception { createRequestedDeltaCommit(basePath, instantTime); createInflightDeltaCommit(basePath, instantTime); @@ -173,6 +186,13 @@ public HoodieTestTable addReplaceCommit(String instantTime, HoodieRequestedRepla return this; } + public HoodieTestTable addRequestedReplace(String instantTime, HoodieRequestedReplaceMetadata requestedReplaceMetadata) throws Exception { + createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addInflightClean(String instantTime, HoodieCleanerPlan cleanerPlan) throws IOException { createRequestedCleanFile(basePath, instantTime, cleanerPlan); createInflightCleanFile(basePath, instantTime, cleanerPlan); @@ -190,6 +210,14 @@ public HoodieTestTable addClean(String instantTime, HoodieCleanerPlan cleanerPla return this; } + public HoodieTestTable addRollback(String instantTime, HoodieRollbackMetadata rollbackMetadata) throws IOException { + createInflightRollbackFile(basePath, instantTime); + createRollbackFile(basePath, instantTime, rollbackMetadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addRequestedCompaction(String instantTime) throws IOException { createRequestedCompaction(basePath, instantTime); currentInstantTime = instantTime; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java index c278770081904..0bb122f0bb7d5 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java @@ -232,4 +232,8 @@ private static boolean waitForServerUp(String hostname, int port, long timeout) } return false; } + + public String connectString() { + return bindIP + ":" + clientPort; + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java index 98535e95238e6..c55e34acfa877 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java @@ -93,4 +93,4 @@ private HoodieWriteStat createWriteStat(String partition, String fileId) { writeStat1.setFileId(fileId); return writeStat1; } -} +} \ No newline at end of file diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index d93c663e8e44a..bf00373774048 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -81,6 +81,10 @@ org.eclipse.jetty * + + org.apache.curator + * + @@ -89,11 +93,23 @@ org.apache.hudi hudi-spark_${scala.binary.version} ${project.version} + + + org.apache.curator + * + + org.apache.hudi hudi-spark-common ${project.version} + + + org.apache.curator + * + + org.apache.hudi @@ -117,9 +133,29 @@ com.databricks spark-avro_${scala.binary.version} + + org.apache.curator + * + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + org.apache.curator + curator-client + ${zk-curator.version} + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + log4j @@ -141,7 +177,6 @@ ${project.version} tests test-jar - test org.apache.hudi @@ -213,6 +248,13 @@ test-jar + + org.apache.curator + curator-test + ${zk-curator.version} + test + + com.fasterxml.jackson.dataformat diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index da7953f982e8b..42bbfbfa6a2a0 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -23,13 +23,13 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieTableType; 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.ReflectionUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.dag.DagUtils; import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator; @@ -38,6 +38,8 @@ import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode; import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler; import org.apache.hudi.integ.testsuite.dag.scheduler.SaferSchemaDagScheduler; +import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider; +import org.apache.hudi.integ.testsuite.helpers.ZookeeperServiceProvider; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; import org.apache.hudi.keygen.BuiltinKeyGenerator; @@ -182,9 +184,9 @@ public void runTestSuite() { long startTime = System.currentTimeMillis(); WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); writerContext.initContext(jsc); + startOtherServicesIfNeeded(writerContext); if (this.cfg.saferSchemaEvolution) { int numRollbacks = 2; // rollback most recent upsert/insert, by default. - // if root is RollbackNode, get num_rollbacks List root = workflowDag.getNodeList(); if (!root.isEmpty() && root.get(0) instanceof RollbackNode) { @@ -203,8 +205,31 @@ public void runTestSuite() { log.error("Failed to run Test Suite ", e); throw new HoodieException("Failed to run Test Suite ", e); } finally { + stopQuietly(); + } + } + + private void stopQuietly() { + try { sparkSession.stop(); jsc.stop(); + } catch (Exception e) { + log.error("Unable to stop spark session", e); + } + } + + private void startOtherServicesIfNeeded(WriterContext writerContext) throws Exception { + if (cfg.startHiveMetastore) { + HiveServiceProvider hiveServiceProvider = new HiveServiceProvider( + Config.newBuilder().withHiveLocal(true).build()); + hiveServiceProvider.startLocalHiveServiceIfNeeded(writerContext.getHoodieTestSuiteWriter().getConfiguration()); + hiveServiceProvider.syncToLocalHiveIfNeeded(writerContext.getHoodieTestSuiteWriter()); + } + + if (cfg.startZookeeper) { + ZookeeperServiceProvider zookeeperServiceProvider = new ZookeeperServiceProvider(Config.newBuilder().withHiveLocal(true).build(), + writerContext.getHoodieTestSuiteWriter().getConfiguration()); + zookeeperServiceProvider.startLocalZookeeperIfNeeded(); } } @@ -262,5 +287,11 @@ public static class HoodieTestSuiteConfig extends HoodieDeltaStreamer.Config { + "(If not provided, assumed to be false.)", required = false) public Boolean saferSchemaEvolution = false; + + @Parameter(names = {"--start-zookeeper"}, description = "Start Zookeeper instance to use for optimistic lock ") + public Boolean startZookeeper = false; + + @Parameter(names = {"--start-hive-metastore"}, description = "Start Hive Metastore to use for optimistic lock ") + public Boolean startHiveMetastore = false; } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index c3294dede6d09..a30312f7fa525 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -18,6 +18,9 @@ package org.apache.hudi.integ.testsuite; +import java.io.Serializable; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; @@ -28,7 +31,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; @@ -40,12 +42,12 @@ import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode; import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode; import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; -import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +62,7 @@ * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are * {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}. */ -public class HoodieTestSuiteWriter { +public class HoodieTestSuiteWriter implements Serializable { private static Logger log = LoggerFactory.getLogger(HoodieTestSuiteWriter.class); @@ -121,6 +123,14 @@ private boolean allowWriteClientAccess(DagNode dagNode) { return false; } + public RDD getNextBatch() throws Exception { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + JavaRDD inputRDD = nextBatch.getRight().getRight(); + return inputRDD.map(r -> (GenericRecord) r.getData() + .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); + } + public Pair>> fetchSource() throws Exception { return this.deltaStreamerWrapper.fetchSource(); } @@ -253,4 +263,12 @@ public JavaSparkContext getSparkContext() { public Option getLastCheckpoint() { return lastCheckpoint; } + + public Properties getProps() { + return props; + } + + public String getSchema() { + return schema; + } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index 650ab1eeaf4c2..27760f7116ecd 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -109,4 +109,8 @@ public TypedProperties getProps() { public String toString() { return this.hoodieTestSuiteWriter.toString() + "\n" + this.deltaGenerator.toString() + "\n"; } + + public SparkSession getSparkSession() { + return sparkSession; + } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.java new file mode 100644 index 0000000000000..afd8a07590587 --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.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.integ.testsuite.helpers; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; + +public class ZookeeperServiceProvider { + + Configuration configuration; + private CuratorFramework client; + private Config config; + + public ZookeeperServiceProvider(Config config, Configuration configuration) { + this.configuration = configuration; + this.config = config; + } + + public void startLocalZookeeperIfNeeded() throws Exception { + if (config.isHiveLocal()) { + ZookeeperTestService zookeeperTestService = new ZookeeperTestService(configuration); + zookeeperTestService.start(); + } + } + + public void stopLocalZookeeperIfNeeded() throws Exception { + if (config.isHiveLocal()) { + ZookeeperTestService zookeeperTestService = new ZookeeperTestService(configuration); + zookeeperTestService.stop(); + } + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java index 4bd096ae0cbde..054870f1fe653 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java @@ -49,7 +49,6 @@ public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputWriter) { public List write(Iterator input) throws IOException { while (input.hasNext()) { GenericRecord next = input.next(); - next.put(SchemaUtils.SOURCE_ORDERING_FIELD, preCombineFieldVal); if (this.deltaInputWriter.canWrite()) { this.deltaInputWriter.writeData(next); } else { diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala new file mode 100644 index 0000000000000..cb25a6d2e1698 --- /dev/null +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala @@ -0,0 +1,65 @@ +/* + * 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.integ.testsuite.dag.nodes + +import org.apache.hudi.client.WriteStatus +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config +import org.apache.hudi.integ.testsuite.dag.ExecutionContext +import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SaveMode + +import scala.collection.JavaConverters._ + +/** + * Spark datasource based insert node + * @param config1 + */ +class SparkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] { + + config = config1 + + /** + * Execute the {@link DagNode}. + * + * @param context The context needed for an execution of a node. + * @param curItrCount iteration count for executing the node. + * @throws Exception Thrown if the execution failed. + */ + override def execute(context: ExecutionContext, curItrCount: Int): Unit = { + if (!config.isDisableGenerate) { + println("Generating input data for node {}", this.getName) + context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count() + } + val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch, + context.getWriterContext.getHoodieTestSuiteWriter.getSchema, + context.getWriterContext.getSparkSession) + inputDF.write.format("hudi") + .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key") + .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse("")) + .option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .mode(SaveMode.Overwrite) + .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath) + } +} diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala new file mode 100644 index 0000000000000..1cc33eda5c5ce --- /dev/null +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala @@ -0,0 +1,65 @@ +/* + * 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.integ.testsuite.dag.nodes + +import org.apache.hudi.client.WriteStatus +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config +import org.apache.hudi.integ.testsuite.dag.ExecutionContext +import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SaveMode + +import scala.collection.JavaConverters._ + +/** + * Spark datasource based upsert node + * @param config1 + */ +class SparkUpsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] { + + config = config1 + + /** + * Execute the {@link DagNode}. + * + * @param context The context needed for an execution of a node. + * @param curItrCount iteration count for executing the node. + * @throws Exception Thrown if the execution failed. + */ + override def execute(context: ExecutionContext, curItrCount: Int): Unit = { + if (!config.isDisableGenerate) { + println("Generating input data for node {}", this.getName) + context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count() + } + val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch, + context.getWriterContext.getHoodieTestSuiteWriter.getSchema, + context.getWriterContext.getSparkSession) + inputDF.write.format("hudi") + .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key") + .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse("")) + .option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .mode(SaveMode.Append) + .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath) + } +} diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index 6232b1dfc2f89..ec470e0231712 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -69,6 +69,8 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase { private static final String MOR_DAG_FILE_NAME = "unit-test-mor-dag.yaml"; private static final String MOR_DAG_SOURCE_PATH = "/hudi-integ-test/src/test/resources/" + MOR_DAG_FILE_NAME; + private static final String COW_DAG_FILE_NAME_SPARK_DATASOURCE_NODES = "unit-test-cow-dag-spark-datasource.yaml"; + private static final String COW_DAG_SPARK_DATASOURCE_NODES_RELATIVE_PATH = "/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml"; public static Stream configParams() { Object[][] data = @@ -92,39 +94,15 @@ public static void initClass() throws Exception { UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.." + MOR_DAG_SOURCE_PATH, dfs, dfsBasePath + "/" + MOR_DAG_FILE_NAME); - TypedProperties props = new TypedProperties(); - props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); - props.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp"); - props.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", "UNIX_TIMESTAMP"); - props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd"); - props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); - props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc"); - props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/input"); - props.setProperty("hoodie.datasource.hive_sync.assume_date_partitioning", "true"); - props.setProperty("hoodie.datasource.hive_sync.skip_ro_suffix", "true"); - props.setProperty("hoodie.datasource.write.keytranslator.class", "org.apache.hudi" - + ".DayBasedPartitionPathKeyTranslator"); - props.setProperty("hoodie.compact.inline.max.delta.commits", "3"); - props.setProperty("hoodie.parquet.max.file.size", "1024000"); - props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); - props.setProperty("hoodie.index.type", HoodieIndex.IndexType.GLOBAL_SIMPLE.name()); - props.setProperty("hoodie.global.simple.index.parallelism", "2"); - // Reduce shuffle parallelism, spark hangs when numPartitions >> numRecords to process - props.setProperty("hoodie.insert.shuffle.parallelism", "10"); - props.setProperty("hoodie.upsert.shuffle.parallelism", "10"); - props.setProperty("hoodie.bulkinsert.shuffle.parallelism", "10"); - props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); - // Make path selection test suite specific - props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); - // Hive Configs - props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1"); - props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "table1"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr"); - props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), TimestampBasedKeyGenerator.class.getName()); + TypedProperties props = getProperties(); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source" + ".properties"); + UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.." + + COW_DAG_SPARK_DATASOURCE_NODES_RELATIVE_PATH, dfs, dfsBasePath + "/" + COW_DAG_FILE_NAME_SPARK_DATASOURCE_NODES); + UtilitiesTestBase.Helpers.savePropsToDFS(getProperties(), dfs, dfsBasePath + "/test-source" + + ".properties"); + // Properties used for the delta-streamer which incrementally pulls from upstream DFS Avro source and // writes to downstream hudi table TypedProperties downstreamProps = new TypedProperties(); @@ -161,6 +139,50 @@ private void cleanDFSDirs() throws Exception { dfs.delete(new Path(dfsBasePath + "/result"), true); } + private static TypedProperties getProperties() { + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + props.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp"); + props.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", "UNIX_TIMESTAMP"); + props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd"); + props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); + props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc"); + props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/input"); + props.setProperty("hoodie.datasource.hive_sync.assume_date_partitioning", "true"); + props.setProperty("hoodie.datasource.hive_sync.skip_ro_suffix", "true"); + props.setProperty("hoodie.datasource.write.keytranslator.class", "org.apache.hudi" + + ".DayBasedPartitionPathKeyTranslator"); + props.setProperty("hoodie.compact.inline.max.delta.commits", "3"); + props.setProperty("hoodie.parquet.max.file.size", "1024000"); + props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); + props.setProperty("hoodie.index.type", HoodieIndex.IndexType.GLOBAL_SIMPLE.name()); + props.setProperty("hoodie.global.simple.index.parallelism", "2"); + // Reduce shuffle parallelism, spark hangs when numPartitions >> numRecords to process + props.setProperty("hoodie.insert.shuffle.parallelism", "10"); + props.setProperty("hoodie.upsert.shuffle.parallelism", "10"); + props.setProperty("hoodie.bulkinsert.shuffle.parallelism", "10"); + props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); + // Make path selection test suite specific + props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); + // Hive Configs + props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1"); + props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "table1"); + props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr"); + props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), TimestampBasedKeyGenerator.class.getName()); + + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); + props.setProperty("hoodie.writer.lock.hivemetastore.database", "testdb1"); + props.setProperty("hoodie.writer.lock.hivemetastore.table", "table1"); + props.setProperty("hoodie.writer.lock.zookeeper.url", "127.0.0.1"); + props.setProperty("hoodie.writer.lock.zookeeper.port", "2828"); + props.setProperty("hoodie.writer.lock.wait_time_ms", "1200000"); + props.setProperty("hoodie.writer.lock.num_retries", "10"); + props.setProperty("hoodie.writer.lock.lock_key", "test_table"); + props.setProperty("hoodie.writer.lock.zookeeper.zk_base_path", "/test"); + return props; + } + // Tests in this class add to the test build time significantly. Since this is a Integration Test (end to end), we // would like to run this as a nightly build which is a TODO. // TODO : Clean up input / result paths after each test @@ -226,6 +248,27 @@ public void testMORFullDagFromYaml() throws Exception { //assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7); } + @Test + public void testSparkDataSourceNodesDagWithLock() throws Exception { + boolean useDeltaStreamer = false; + this.cleanDFSDirs(); + + TypedProperties props = getProperties(); + props.setProperty("hoodie.write.concurrency.mode", "optimistic_concurrency_control"); + props.setProperty("hoodie.failed.writes.cleaner.policy", "LAZY"); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source" + + ".properties"); + String inputBasePath = dfsBasePath + "/input"; + String outputBasePath = dfsBasePath + "/result"; + HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, HoodieTableType + .COPY_ON_WRITE.name()); + cfg.workloadYamlPath = dfsBasePath + "/" + COW_DAG_FILE_NAME_SPARK_DATASOURCE_NODES; + HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); + hoodieTestSuiteJob.runTestSuite(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); + assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 3); + } + protected HoodieTestSuiteConfig makeConfig(String inputBasePath, String outputBasePath, boolean useDeltaStream, String tableType) { HoodieTestSuiteConfig cfg = new HoodieTestSuiteConfig(); diff --git a/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml b/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml new file mode 100644 index 0000000000000..16023f69ed5e8 --- /dev/null +++ b/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml @@ -0,0 +1,43 @@ +# 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. +dag_name: unit-test-cow-dag-spark-datasource +dag_rounds: 1 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 100 + type: SparkInsertNode + deps: none + first_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: SparkUpsertNode + deps: first_insert + second_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: SparkUpsertNode + deps: first_upsert \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index d2c9485020c53..171513138c479 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -203,6 +203,12 @@ org.apache.hudi hudi-spark-common ${project.version} + + + org.apache.curator + * + + org.apache.hudi @@ -379,6 +385,24 @@ + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + org.apache.hudi diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 1f1dc4d4233bf..8c1716fb3c53b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -497,8 +497,8 @@ private[hudi] object HoodieSparkSqlWriter { private def isAsyncCompactionEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]], tableConfig: HoodieTableConfig, parameters: Map[String, String], configuration: Configuration) : Boolean = { - log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}") - if (asyncCompactionTriggerFnDefined && !client.getConfig.isInlineCompaction + log.info(s"Config.inlineCompactionEnabled ? ${client.getConfig.inlineCompactionEnabled}") + if (asyncCompactionTriggerFnDefined && !client.getConfig.inlineCompactionEnabled && parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY).exists(r => r.toBoolean)) { tableConfig.getTableType == HoodieTableType.MERGE_ON_READ } else { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java new file mode 100644 index 0000000000000..4a96c7890c51e --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java @@ -0,0 +1,226 @@ +/* + * 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.hive; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.thrift.TException; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.lock.LockState.ACQUIRING; +import static org.apache.hudi.common.lock.LockState.ALREADY_ACQUIRED; +import static org.apache.hudi.common.lock.LockState.FAILED_TO_ACQUIRE; +import static org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE; +import static org.apache.hudi.common.lock.LockState.RELEASED; +import static org.apache.hudi.common.lock.LockState.RELEASING; + +/** + * A hivemetastore based lock. Default HiveMetastore Lock Manager uses zookeeper to provide locks, read here + * {@link /cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-Locking} + * This {@link LockProvider} implementation allows to lock table operations + * using hive metastore APIs. Users need to have a HiveMetastore & Zookeeper cluster deployed to be able to use this lock. + * + */ +public class HiveMetastoreLockProvider implements LockProvider { + + private static final Logger LOG = LogManager.getLogger(HiveMetastoreLockProvider.class); + + private final String databaseName; + private final String tableName; + private IMetaStoreClient hiveClient; + private volatile LockResponse lock = null; + protected LockConfiguration lockConfiguration; + ExecutorService executor = Executors.newSingleThreadExecutor(); + + public HiveMetastoreLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + this(lockConfiguration); + try { + HiveConf hiveConf = new HiveConf(); + setHiveLockConfs(hiveConf); + hiveConf.addResource(conf); + this.hiveClient = Hive.get(hiveConf).getMSC(); + } catch (MetaException | HiveException e) { + throw new HoodieLockException("Failed to create HiveMetaStoreClient", e); + } + } + + public HiveMetastoreLockProvider(final LockConfiguration lockConfiguration, final IMetaStoreClient metaStoreClient) { + this(lockConfiguration); + this.hiveClient = metaStoreClient; + } + + HiveMetastoreLockProvider(final LockConfiguration lockConfiguration) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.databaseName = this.lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP); + this.tableName = this.lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP); + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + } catch (ExecutionException | InterruptedException | TimeoutException | TException e) { + throw new HoodieLockException(generateLogStatement(FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } + return this.lock != null && this.lock.getState() == LockState.ACQUIRED; + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(RELEASING, generateLogSuffixString())); + LockResponse lockResponseLocal = lock; + if (lockResponseLocal == null) { + return; + } + lock = null; + hiveClient.unlock(lockResponseLocal.getLockid()); + LOG.info(generateLogStatement(RELEASED, generateLogSuffixString())); + } catch (TException e) { + throw new HoodieLockException(generateLogStatement(FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + public void acquireLock(long time, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException, TException { + ValidationUtils.checkArgument(this.lock == null, ALREADY_ACQUIRED.name()); + final LockComponent lockComponent = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, this.databaseName); + lockComponent.setTablename(tableName); + acquireLockInternal(time, unit, lockComponent); + } + + // NOTE: HiveMetastoreClient does not implement AutoCloseable. Additionally, we cannot call close() after unlock() + // because if there are multiple operations started from the same WriteClient (via multiple threads), closing the + // hive client causes all other threads who may have already initiated the tryLock() to fail since the + // HiveMetastoreClient is shared. + @Override + public void close() { + try { + if (lock != null) { + hiveClient.unlock(lock.getLockid()); + } + hiveClient.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + public IMetaStoreClient getHiveClient() { + return hiveClient; + } + + @Override + public LockResponse getLock() { + return this.lock; + } + + // This API is exposed for tests and not intended to be used elsewhere + public boolean acquireLock(long time, TimeUnit unit, final LockComponent component) + throws InterruptedException, ExecutionException, TimeoutException, TException { + ValidationUtils.checkArgument(this.lock == null, ALREADY_ACQUIRED.name()); + acquireLockInternal(time, unit, component); + return this.lock != null && this.lock.getState() == LockState.ACQUIRED; + } + + private void acquireLockInternal(long time, TimeUnit unit, LockComponent lockComponent) + throws InterruptedException, ExecutionException, TimeoutException, TException { + LockRequest lockRequest = null; + try { + final LockRequestBuilder builder = new LockRequestBuilder("hudi-lock"); + lockRequest = builder.addLockComponent(lockComponent).setUser(System.getProperty("user.name")).build(); + lockRequest.setUserIsSet(true); + final LockRequest lockRequestFinal = lockRequest; + this.lock = executor.submit(() -> hiveClient.lock(lockRequestFinal)) + .get(time, unit); + } catch (InterruptedException | TimeoutException e) { + if (this.lock != null && this.lock.getState() == LockState.ACQUIRED) { + return; + } else if (lockRequest != null) { + LockResponse lockResponse = this.hiveClient.checkLock(lockRequest.getTxnid()); + if (lockResponse.getState() == LockState.ACQUIRED) { + this.lock = lockResponse; + return; + } else { + throw e; + } + } else { + throw e; + } + } + } + + private void checkRequiredProps(final LockConfiguration lockConfiguration) { + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP) != null); + } + + private void setHiveLockConfs(HiveConf hiveConf) { + hiveConf.set("hive.support.concurrency", "true"); + hiveConf.set("hive.lock.manager", "org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager"); + hiveConf.set("hive.lock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP)); + hiveConf.set("hive.unlock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP)); + hiveConf.set("hive.lock.sleep.between.retries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP)); + hiveConf.set("hive.zookeeper.quorum", lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)); + hiveConf.set("hive.zookeeper.client.port", lockConfiguration.getConfig().getString(ZK_PORT_PROP)); + hiveConf.set("hive.zookeeper.session.timeout", lockConfiguration.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP)); + } + + private String generateLogSuffixString() { + return StringUtils.join(" database ", databaseName, " and ", "table ", tableName); + } + + protected String generateLogStatement(org.apache.hudi.common.lock.LockState state, String suffix) { + return StringUtils.join(state.name(), " lock at", suffix); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java new file mode 100644 index 0000000000000..51f7814c3d1e5 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java @@ -0,0 +1,158 @@ +/* + * 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.hive; + +import org.apache.hadoop.hive.metastore.api.DataOperationType; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.hive.testutils.HiveTestUtil; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * For all tests, we need to set LockComponent.setOperationType(DataOperationType.NO_TXN). + * This is needed because of this -> https://github.com/apache/hive/blob/master/standalone-metastore + * /metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java#L2892 + * Unless this is set, we cannot use HiveMetastore server in tests for locking use-cases. + */ +public class TestHiveMetastoreLockProvider { + + private static Connection connection; + private static LockComponent lockComponent = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, "testdb"); + private static LockConfiguration lockConfiguration; + + @BeforeAll + public static void init() throws Exception { + HiveTestUtil.setUp(); + createHiveConnection(); + connection.createStatement().execute("create database if not exists testdb"); + TypedProperties properties = new TypedProperties(); + properties.setProperty(HIVE_DATABASE_NAME_PROP, "testdb"); + properties.setProperty(HIVE_TABLE_NAME_PROP, "testtable"); + properties.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES); + properties.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS); + properties.setProperty(ZK_CONNECT_URL_PROP, HiveTestUtil.getZkService().connectString()); + properties.setProperty(ZK_PORT_PROP, HiveTestUtil.getHiveConf().get("hive.zookeeper.client.port")); + properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, HiveTestUtil.getHiveConf().get("hive.zookeeper.session.timeout")); + properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS)); + properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(1000)); + lockConfiguration = new LockConfiguration(properties); + lockComponent.setTablename("testtable"); + } + + @AfterAll + public static void cleanUpClass() { + HiveTestUtil.shutdown(); + } + + @Test + public void testAcquireLock() throws Exception { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + try { + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + Assertions.fail(); + } catch (Exception e) { + // Expected since lock is already acquired + } + lockProvider.unlock(); + // try to lock again after unlocking + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + lockProvider.close(); + } + + @Test + public void testUnlock() throws Exception { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + lockProvider.unlock(); + // try to lock again after unlocking + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + lockProvider.close(); + } + + @Test + public void testReentrantLock() throws Exception { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + try { + lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent); + Assertions.fail(); + } catch (IllegalArgumentException e) { + // expected + } + lockProvider.unlock(); + } + + @Test + public void testUnlockWithoutLock() { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + lockProvider.unlock(); + } + + private static void createHiveConnection() { + if (connection == null) { + try { + Class.forName("org.apache.hive.jdbc.HiveDriver"); + } catch (ClassNotFoundException e) { + throw new RuntimeException(); + } + try { + connection = DriverManager.getConnection("jdbc:hive2://127.0.0.1:9999/"); + } catch (SQLException e) { + throw new HoodieHiveSyncException("Cannot create hive connection ", e); + } + } + } + +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java index 2090b65de9062..ca7aa7a02ce49 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java @@ -148,7 +148,11 @@ public void stop() { hadoopConf = null; } - private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { + public HiveServer2 getHiveServer() { + return hiveServer; + } + + public HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { conf.set("hive.metastore.local", "false"); conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort); conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 5feca25b134be..80a681db40e07 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -86,6 +86,7 @@ public class HiveTestUtil { private static ZooKeeperServer zkServer; private static HiveServer2 hiveServer; private static HiveTestService hiveTestService; + private static ZookeeperTestService zkService; private static Configuration configuration; public static HiveSyncConfig hiveSyncConfig; private static DateTimeFormatter dtfOut; @@ -99,7 +100,7 @@ public static void setUp() throws IOException, InterruptedException { configuration = service.getHadoopConf(); } if (zkServer == null) { - ZookeeperTestService zkService = new ZookeeperTestService(configuration); + zkService = new ZookeeperTestService(configuration); zkServer = zkService.start(); } if (hiveServer == null) { @@ -145,6 +146,18 @@ public static HiveConf getHiveConf() { return hiveServer.getHiveConf(); } + public static HiveServer2 getHiveServer() { + return hiveServer; + } + + public static ZooKeeperServer getZkServer() { + return zkServer; + } + + public static ZookeeperTestService getZkService() { + return zkService; + } + public static void shutdown() { if (hiveServer != null) { hiveServer.stop(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 17eee8ee5c41d..336639c98a4a8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -650,7 +650,7 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { } // Validate what deltastreamer assumes of write-config to be really safe - ValidationUtils.checkArgument(config.isInlineCompaction() == cfg.isInlineCompactionEnabled(), + ValidationUtils.checkArgument(config.inlineCompactionEnabled() == cfg.isInlineCompactionEnabled(), String.format("%s should be set to %s", INLINE_COMPACT_PROP, cfg.isInlineCompactionEnabled())); ValidationUtils.checkArgument(!config.shouldAutoCommit(), String.format("%s should be set to %s", HOODIE_AUTO_COMMIT_PROP, autoCommit)); diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index db2635af2603a..c24021568bdc7 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -153,6 +153,9 @@ com.fasterxml.jackson.dataformat:jackson-dataformat-yaml org.apache.htrace:htrace-core + org.apache.curator:curator-framework + org.apache.curator:curator-client + org.apache.curator:curator-recipes @@ -641,5 +644,24 @@ 0.9.3 + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index ba91f54987500..e20123e5c664b 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -112,7 +112,10 @@ org.apache.hbase:hbase-protocol org.apache.hbase:hbase-server org.apache.htrace:htrace-core - commons-codec:commons-codec + org.apache.curator:curator-framework + org.apache.curator:curator-client + org.apache.curator:curator-recipes + commons-codec:commons-codec @@ -352,6 +355,25 @@ ${hbase.version} + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + diff --git a/pom.xml b/pom.xml index 569ce7b50e55a..ad726c0d76b42 100644 --- a/pom.xml +++ b/pom.xml @@ -143,6 +143,7 @@ compile org.apache.hudi. true + 2.7.1 @@ -907,6 +908,25 @@ test + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + org.junit.jupiter junit-jupiter-api From 0e7fbb3ab8c02fbaee04adfc89205d6bee0834ab Mon Sep 17 00:00:00 2001 From: Nishith Date: Sun, 14 Mar 2021 21:09:56 -0700 Subject: [PATCH 2/2] Adding tests for Deltastreamer, fixing some documentation and added metadata overriding --- .../client/AbstractHoodieWriteClient.java | 108 ++++--- ...tFileWritesConflictResolutionStrategy.java | 2 - .../client/utils/MetadataConversionUtils.java | 8 +- .../hudi/client/utils/TransactionUtils.java | 57 ++++ .../apache/hudi/config/HoodieWriteConfig.java | 17 +- .../commit/BaseCommitActionExecutor.java | 8 +- .../FileSystemBasedLockProviderTestClass.java | 17 +- .../hudi/client/HoodieFlinkWriteClient.java | 10 - .../minicluster/ZookeeperTestService.java | 4 +- .../functional/TestHoodieDeltaStreamer.java | 290 +++++++++++++++++- .../testutils/UtilitiesTestBase.java | 7 + .../sources/AbstractBaseTestSource.java | 3 + packaging/hudi-utilities-bundle/pom.xml | 22 ++ 13 files changed, 461 insertions(+), 92 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 656b209e78042..9f3b74b0c4708 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -33,6 +33,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.heartbeat.HeartbeatUtils; 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.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -107,7 +108,7 @@ public abstract class AbstractHoodieWriteClient lastCompletedTxn; + protected Option>> lastCompletedTxnAndMetadata = Option.empty(); /** * Create a write client, with new hudi index. @@ -179,20 +180,20 @@ public boolean commitStats(String instantTime, List stats, Opti finalizeWrite(table, instantTime, stats); HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config); this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime)), - lastCompletedTxn); + lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); try { preCommit(instantTime, metadata); commit(table, commitActionType, instantTime, metadata, stats); postCommit(table, metadata, instantTime, extraMetadata); - emitCommitMetrics(instantTime, metadata, commitActionType); LOG.info("Committed " + instantTime); } catch (IOException e) { - throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, e); } finally { this.txnManager.endTransaction(); } - + // do this outside of lock since compaction, clustering can be time taking and we don't need a lock for the entire execution period + runTableServicesInline(table, metadata, extraMetadata); + emitCommitMetrics(instantTime, metadata, commitActionType); // callback if needed. if (config.writeCommitCallbackOn()) { if (null == commitCallback) { @@ -392,9 +393,10 @@ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instan protected void preWrite(String instantTime, WriteOperationType writeOperationType, HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); - this.lastCompletedTxn = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .lastInstant(); - this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxn); + this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient); + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxnAndMetadata + .isPresent() + ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); try { syncTableMetadata(); } finally { @@ -422,29 +424,8 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp */ protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { - // Delete the marker directory for the instant. new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - - if (config.inlineTableServices()) { - // Do an inline compaction if enabled - if (config.inlineCompactionEnabled()) { - runAnyPendingCompactions(table); - metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); - inlineCompact(extraMetadata); - } else { - metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); - } - - // Do an inline clustering if enabled - if (config.inlineClusteringEnabled()) { - runAnyPendingClustering(table); - metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"); - inlineCluster(extraMetadata); - } else { - metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false"); - } - } // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); @@ -457,6 +438,28 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me } } + protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { + if (config.inlineTableServices()) { + // Do an inline compaction if enabled + if (config.inlineCompactionEnabled()) { + runAnyPendingCompactions(table); + metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); + inlineCompact(extraMetadata); + } else { + metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); + } + + // Do an inline clustering if enabled + if (config.inlineClusteringEnabled()) { + runAnyPendingClustering(table); + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"); + inlineCluster(extraMetadata); + } else { + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false"); + } + } + } + protected void runAnyPendingCompactions(HoodieTable table) { table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants() .forEach(instant -> { @@ -719,7 +722,7 @@ private void startCommit(String instantTime, String actionType, HoodieTableMetaC */ public Option scheduleCompaction(Option> extraMetadata) throws HoodieIOException { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleTableService(instantTime, extraMetadata, TableServiceType.COMPACT); + return scheduleCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); } /** @@ -728,10 +731,7 @@ public Option scheduleCompaction(Option> extraMetada * @param extraMetadata Extra Metadata to be stored */ public boolean scheduleCompactionAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { - LOG.info("Scheduling compaction at instant time :" + instantTime); - Option plan = createTable(config, hadoopConf) - .scheduleCompaction(context, instantTime, extraMetadata); - return plan.isPresent(); + return scheduleTableService(instantTime, extraMetadata, TableServiceType.COMPACT).isPresent(); } /** @@ -850,9 +850,7 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H * Performs a compaction operation on a table, serially before or after an insert/upsert action. */ protected Option inlineCompact(Option> extraMetadata) { - String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime(); - Option compactionInstantTimeOpt = scheduleTableServiceInternal(compactionInstantTime, - extraMetadata, TableServiceType.COMPACT); + Option compactionInstantTimeOpt = scheduleCompaction(extraMetadata); compactionInstantTimeOpt.ifPresent(compactInstantTime -> { // inline compaction should auto commit as the user is never given control compact(compactInstantTime, true); @@ -866,7 +864,7 @@ protected Option inlineCompact(Option> extraMetadata */ public Option scheduleClustering(Option> extraMetadata) throws HoodieIOException { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLUSTER); + return scheduleClusteringAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); } /** @@ -875,10 +873,7 @@ public Option scheduleClustering(Option> extraMetada * @param extraMetadata Extra Metadata to be stored */ public boolean scheduleClusteringAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { - LOG.info("Scheduling clustering at instant time :" + instantTime); - Option plan = createTable(config, hadoopConf) - .scheduleClustering(context, instantTime, extraMetadata); - return plan.isPresent(); + return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLUSTER).isPresent(); } /** @@ -887,7 +882,7 @@ public boolean scheduleClusteringAtInstant(String instantTime, Option scheduleCleaning(Option> extraMetadata) throws HoodieIOException { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLEAN); + return scheduleCleaningAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); } /** @@ -896,15 +891,11 @@ protected Option scheduleCleaning(Option> extraMetad * @param extraMetadata Extra Metadata to be stored */ protected boolean scheduleCleaningAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { - LOG.info("Scheduling clustering at instant time :" + instantTime); - Option plan = createTable(config, hadoopConf) - .scheduleCleaning(context, instantTime, extraMetadata); - return plan.isPresent(); + return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLEAN).isPresent(); } /** * Ensures clustering instant is in expected state and performs clustering for the plan stored in metadata. - * * @param clusteringInstant Clustering Instant Time * @return Collection of Write Status */ @@ -946,11 +937,20 @@ private Option scheduleTableServiceInternal(String instantTime, Option clusteringPlan = createTable(config, hadoopConf) + .scheduleClustering(context, instantTime, extraMetadata); + return clusteringPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case COMPACT: - return scheduleCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + LOG.info("Scheduling compaction at instant time :" + instantTime); + Option compactionPlan = createTable(config, hadoopConf) + .scheduleCompaction(context, instantTime, extraMetadata); + return compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case CLEAN: - return scheduleCleaningAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + LOG.info("Scheduling cleaning at instant time :" + instantTime); + Option cleanerPlan = createTable(config, hadoopConf) + .scheduleCleaning(context, instantTime, extraMetadata); + return cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty(); default: throw new IllegalArgumentException("Invalid TableService " + tableServiceType); } @@ -960,9 +960,7 @@ private Option scheduleTableServiceInternal(String instantTime, Option inlineCluster(Option> extraMetadata) { - String clusteringInstantTme = HoodieActiveTimeline.createNewInstantTime(); - Option clusteringInstantOpt = scheduleTableServiceInternal(clusteringInstantTme, - extraMetadata, TableServiceType.CLUSTER); + Option clusteringInstantOpt = scheduleClustering(extraMetadata); clusteringInstantOpt.ifPresent(clusteringInstant -> { // inline cluster should auto commit as the user is never given control cluster(clusteringInstant, true); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java index e7cad4452501b..938a40684a092 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -55,8 +55,6 @@ public Stream getCandidateInstants(HoodieActiveTimeline activeTim // 2. Get any scheduled or completed compaction or clustering operations that have started and/or finished // after the current instant. We need to check for write conflicts since they may have mutated the same files // that are being newly created by the current write. - // NOTE that any commits from table services such as compaction, clustering or cleaning since the - // overlapping of files is handled using MVCC. Stream completedCommitsInstantStream = activeTimeline .getCommitsTimeline() .filterCompletedInstants() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java index 49b93182a8c67..b46f3d8660a46 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java @@ -62,7 +62,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst case HoodieTimeline.COMMIT_ACTION: case HoodieTimeline.DELTA_COMMIT_ACTION: { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; @@ -70,7 +70,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst case HoodieTimeline.REPLACE_COMMIT_ACTION: { if (hoodieInstant.isCompleted()) { HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); archivedMetaWrapper.setHoodieReplaceCommitMetadata(ReplaceArchivalHelper.convertReplaceCommitMetadata(replaceCommitMetadata)); } else { HoodieRequestedReplaceMetadata requestedReplaceMetadata = @@ -82,13 +82,13 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst } case HoodieTimeline.ROLLBACK_ACTION: { archivedMetaWrapper.setHoodieRollbackMetadata(TimelineMetadataUtils.deserializeAvroMetadata( - metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); + metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); archivedMetaWrapper.setActionType(ActionType.rollback.name()); break; } case HoodieTimeline.SAVEPOINT_ACTION: { archivedMetaWrapper.setHoodieSavePointMetadata(TimelineMetadataUtils.deserializeAvroMetadata( - metaClient.getCommitTimeline().getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class)); + metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class)); archivedMetaWrapper.setActionType(ActionType.savepoint.name()); break; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java index d4ba4fb0ddadd..80a412010203e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java @@ -18,12 +18,20 @@ package org.apache.hudi.client.utils; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.hudi.client.transaction.ConflictResolutionStrategy; import org.apache.hudi.client.transaction.ConcurrentOperation; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; 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.exception.HoodieIOException; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; @@ -64,8 +72,57 @@ public static Option resolveWriteConflictIfAny(final Hoodi } }); LOG.info("Successfully resolved conflicts, if any"); + // carry over necessary metadata from latest commit metadata + overrideWithLatestCommitMetadata(table.getMetaClient(), thisOperation.getCommitMetadataOption(), currentTxnOwnerInstant, Arrays.asList(config.getWriteMetaKeyPrefixes().split(","))); return thisOperation.getCommitMetadataOption(); } return thisCommitMetadata; } + + /** + * Get the last completed transaction hoodie instant and {@link HoodieCommitMetadata#getExtraMetadata()}. + * @param metaClient + * @return + */ + public static Option>> getLastCompletedTxnInstantAndMetadata( + HoodieTableMetaClient metaClient) { + Option hoodieInstantOption = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); + try { + if (hoodieInstantOption.isPresent()) { + switch (hoodieInstantOption.get().getAction()) { + case HoodieTimeline.REPLACE_COMMIT_ACTION: + HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstantOption.get()).get(), HoodieReplaceCommitMetadata.class); + return Option.of(Pair.of(hoodieInstantOption.get(), replaceCommitMetadata.getExtraMetadata())); + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.COMMIT_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstantOption.get()).get(), HoodieCommitMetadata.class); + return Option.of(Pair.of(hoodieInstantOption.get(), commitMetadata.getExtraMetadata())); + default: + throw new IllegalArgumentException("Unknown instant action" + hoodieInstantOption.get().getAction()); + } + } else { + return Option.empty(); + } + } catch (IOException io) { + throw new HoodieIOException("Unable to read metadata for instant " + hoodieInstantOption.get(), io); + } + } + + // override the current metadata with the metadata from the latest instant for the specified key prefixes + private static void overrideWithLatestCommitMetadata(HoodieTableMetaClient metaClient, Option thisMetadata, + Option thisInstant, List keyPrefixes) { + if (keyPrefixes.size() == 1 && keyPrefixes.get(0).length() < 1) { + return; + } + Option>> lastInstant = getLastCompletedTxnInstantAndMetadata(metaClient); + if (lastInstant.isPresent() && thisMetadata.isPresent()) { + Stream keys = thisMetadata.get().getExtraMetadata().keySet().stream(); + keyPrefixes.stream().forEach(keyPrefix -> keys + .filter(key -> key.startsWith(keyPrefix)) + .forEach(key -> thisMetadata.get().getExtraMetadata().put(key, lastInstant.get().getRight().get(key)))); + } + } } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index f7ad3a0faca1b..944cd02306b36 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -157,6 +157,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { "hoodie.write.concurrency.mode"; public static final String DEFAULT_WRITE_CONCURRENCY_MODE = WriteConcurrencyMode.SINGLE_WRITER.name(); + // Comma separated metadata key prefixes to override from latest commit during overlapping commits via multi writing + public static final String WRITE_META_KEY_PREFIXES_PROP = + "hoodie.write.meta.key.prefixes"; + public static final String DEFAULT_WRITE_META_KEY_PREFIXES = ""; + /** * HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow * multiple write operations (upsert/buk-insert/...) to be executed within a single commit. @@ -1023,6 +1028,10 @@ public Boolean inlineTableServices() { return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean(); } + public String getWriteMetaKeyPrefixes() { + return props.getProperty(WRITE_META_KEY_PREFIXES_PROP); + } + public static class Builder { protected final Properties props = new Properties(); @@ -1307,6 +1316,11 @@ public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) { return this; } + public Builder withWriteMetaKeyPrefixes(String writeMetaKeyPrefixes) { + props.setProperty(WRITE_META_KEY_PREFIXES_PROP, writeMetaKeyPrefixes); + return this; + } + public Builder withProperties(Properties properties) { this.props.putAll(properties); return this; @@ -1369,7 +1383,8 @@ protected void setDefaults() { CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES)); setDefaultOnCondition(props, !props.containsKey(WRITE_CONCURRENCY_MODE_PROP), WRITE_CONCURRENCY_MODE_PROP, DEFAULT_WRITE_CONCURRENCY_MODE); - + setDefaultOnCondition(props, !props.containsKey(WRITE_META_KEY_PREFIXES_PROP), + WRITE_META_KEY_PREFIXES_PROP, DEFAULT_WRITE_META_KEY_PREFIXES); // Make sure the props is propagated setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build()); setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build()); 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 ffb403b213f79..16638fd95642b 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 @@ -33,6 +33,7 @@ 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.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; @@ -62,7 +63,7 @@ public abstract class BaseCommitActionExecutor lastCompletedTxn; + protected Option>> lastCompletedTxn; public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, WriteOperationType operationType, @@ -73,8 +74,7 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c 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 = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .lastInstant(); + this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); } public abstract HoodieWriteMetadata execute(I inputRecords); @@ -134,7 +134,7 @@ protected void commitOnAutoCommit(HoodieWriteMetadata result) { protected void autoCommit(Option> extraMetadata, HoodieWriteMetadata result) { this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime)), - lastCompletedTxn); + lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); try { TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner()); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java index 1a923650d8409..ed0b83b545a04 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieLockException; @@ -47,13 +48,9 @@ public class FileSystemBasedLockProviderTestClass implements LockProvider>, List, List> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { if (operationType == WriteOperationType.DELETE) { setWriteSchemaForDeletes(metaClient); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java index 0bb122f0bb7d5..e5c228f40432b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java @@ -124,14 +124,14 @@ public ZooKeeperServer start() throws IOException, InterruptedException { return zooKeeperServer; } - public void stop() throws IOException { + public void stop() throws RuntimeException { if (!started) { return; } standaloneServerFactory.shutdown(); if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) { - throw new IOException("Waiting for shutdown of standalone server"); + throw new RuntimeException("Waiting for shutdown of standalone server"); } // clear everything diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 7522c2d29474b..8f3e045217cd7 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -18,6 +18,8 @@ package org.apache.hudi.utilities.functional; +import java.util.ConcurrentModificationException; +import java.util.concurrent.ExecutorService; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.TypedProperties; @@ -79,6 +81,7 @@ import org.apache.spark.streaming.kafka010.KafkaTestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -100,6 +103,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -120,6 +124,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { private static final String PROPS_FILENAME_TEST_CSV = "test-csv-dfs-source.properties"; protected static final String PROPS_FILENAME_TEST_PARQUET = "test-parquet-dfs-source.properties"; private static final String PROPS_FILENAME_TEST_JSON_KAFKA = "test-json-kafka-dfs-source.properties"; + private static final String PROPS_FILENAME_TEST_MULTI_WRITER = "test-multi-writer.properties"; private static final String FIRST_PARQUET_FILE_NAME = "1.parquet"; private static String PARQUET_SOURCE_ROOT; private static String JSON_KAFKA_SOURCE_ROOT; @@ -270,6 +275,34 @@ protected static void populateCommonHiveProps(TypedProperties props) { MultiPartKeysValueExtractor.class.getName()); } + protected static TypedProperties prepareMultiWriterProps(String propsFileName) throws IOException { + TypedProperties props = new TypedProperties(); + populateAllCommonProps(props); + + props.setProperty("include", "sql-transformer.properties"); + props.setProperty("hoodie.datasource.write.keygenerator.class", TestGenerator.class.getName()); + props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there"); + props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); + props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc"); + + props.setProperty("include", "base.properties"); + props.setProperty("hoodie.write.concurrency.mode", "optimistic_concurrency_control"); + props.setProperty("hoodie.failed.writes.cleaner.policy", "LAZY"); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); + props.setProperty("hoodie.writer.lock.hivemetastore.database", "testdb1"); + props.setProperty("hoodie.writer.lock.hivemetastore.table", "table1"); + props.setProperty("hoodie.writer.lock.zookeeper.url", "127.0.0.1"); + props.setProperty("hoodie.writer.lock.zookeeper.port", "2828"); + props.setProperty("hoodie.writer.lock.wait_time_ms", "1200000"); + props.setProperty("hoodie.writer.lock.num_retries", "10"); + props.setProperty("hoodie.writer.lock.lock_key", "test_table"); + props.setProperty("hoodie.writer.lock.zookeeper.zk_base_path", "/test"); + + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + propsFileName); + return props; + } + @AfterAll public static void cleanupClass() { UtilitiesTestBase.cleanupClass(); @@ -398,6 +431,22 @@ static void assertAtleastNDeltaCommits(int minExpected, String tablePath, FileSy assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } + static void assertAtleastNCompactionCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numCompactionCommits = (int) timeline.getInstants().count(); + assertTrue(minExpected <= numCompactionCommits, "Got=" + numCompactionCommits + ", exp >=" + minExpected); + } + + static void assertAtleastNDeltaCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numDeltaCommits = (int) timeline.getInstants().count(); + assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); + } + static String assertCommitMetadata(String expected, String tablePath, FileSystem fs, int totalCommits) throws IOException { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); @@ -406,14 +455,14 @@ static String assertCommitMetadata(String expected, String tablePath, FileSystem HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class); assertEquals(totalCommits, timeline.countInstants()); - assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY)); + assertEquals(expected, commitMetadata.getMetadata(CHECKPOINT_KEY)); return lastInstant.getTimestamp(); } - static void waitTillCondition(Function condition, long timeoutInSecs) throws Exception { + static void waitTillCondition(Function condition, Future dsFuture, long timeoutInSecs) throws Exception { Future res = Executors.newSingleThreadExecutor().submit(() -> { boolean ret = false; - while (!ret) { + while (!ret && !dsFuture.isDone()) { try { Thread.sleep(3000); ret = condition.apply(true); @@ -649,6 +698,21 @@ public void testUpsertsMORContinuousMode() throws Exception { testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor"); } + @Test + public void testUpsertsCOWContinuousModeWithMultipleWriters() throws Exception { + testUpsertsContinuousModeWithMultipleWriters(HoodieTableType.COPY_ON_WRITE, "continuous_cow_mulitwriter"); + } + + @Test + public void testUpsertsMORContinuousModeWithMultipleWriters() throws Exception { + testUpsertsContinuousModeWithMultipleWriters(HoodieTableType.MERGE_ON_READ, "continuous_mor_mulitwriter"); + } + + @Test + public void testLatestCheckpointCarryOverWithMultipleWriters() throws Exception { + testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType.COPY_ON_WRITE, "continuous_cow_checkpoint"); + } + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception { String tableBasePath = dfsBasePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode @@ -673,6 +737,215 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir }); } + private void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableType, String tempDir) throws Exception { + // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts + String tableBasePath = dfsBasePath + "/" + tempDir; + // enable carrying forward latest checkpoint + TypedProperties props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + HoodieDeltaStreamer.Config cfgIngestionJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgIngestionJob.continuousMode = true; + cfgIngestionJob.tableType = tableType.name(); + cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer ingestionJob = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + + // Prepare base dataset with some commits + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, (r) -> { + if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { + TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, dfs); + TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, dfs); + } else { + TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, dfs); + } + TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + return true; + }); + + // create a backfill job + HoodieDeltaStreamer.Config cfgBackfillJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgBackfillJob.continuousMode = false; + cfgBackfillJob.tableType = tableType.name(); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc); + + // re-init ingestion job to start sync service + HoodieDeltaStreamer ingestionJob2 = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + + // run ingestion & backfill in parallel, create conflict and fail one + runJobsInParallel(tableBasePath, tableType, totalRecords, ingestionJob2, + cfgIngestionJob, backfillJob, cfgBackfillJob, true); + + // create new ingestion & backfill job config to generate only INSERTS to avoid conflict + props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + props.setProperty("hoodie.test.source.generate.inserts", "true"); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + cfgBackfillJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, + Arrays.asList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgBackfillJob.continuousMode = false; + cfgBackfillJob.tableType = tableType.name(); + meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + + cfgIngestionJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgIngestionJob.continuousMode = true; + cfgIngestionJob.tableType = tableType.name(); + cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + // re-init ingestion job + HoodieDeltaStreamer ingestionJob3 = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + // re-init backfill job + HoodieDeltaStreamer backfillJob2 = new HoodieDeltaStreamer(cfgBackfillJob, jsc); + + // run ingestion & backfill in parallel, avoid conflict and succeed both + runJobsInParallel(tableBasePath, tableType, totalRecords, ingestionJob3, + cfgIngestionJob, backfillJob2, cfgBackfillJob, false); + } + + private void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType, String tempDir) throws Exception { + // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts + String tableBasePath = dfsBasePath + "/" + tempDir; + // enable carrying forward latest checkpoint + TypedProperties props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + HoodieDeltaStreamer.Config cfgIngestionJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgIngestionJob.continuousMode = true; + cfgIngestionJob.tableType = tableType.name(); + cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer ingestionJob = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + + // Prepare base dataset with some commits + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, (r) -> { + if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { + TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, dfs); + TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, dfs); + } else { + TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, dfs); + } + TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + return true; + }); + + // create a backfill job with checkpoint from the first instant + HoodieDeltaStreamer.Config cfgBackfillJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgBackfillJob.continuousMode = false; + cfgBackfillJob.tableType = tableType.name(); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieCommitMetadata commitMetadataForFirstInstant = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + + // get current checkpoint after preparing base dataset with some commits + HoodieCommitMetadata commitMetadataForLastInstant = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + String lastCheckpointBeforeParallelBackfill = commitMetadataForLastInstant.getMetadata(CHECKPOINT_KEY); + + // run the backfill job, enable overriding checkpoint from the latest commit + props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + props.setProperty("hoodie.write.meta.key.prefixes", CHECKPOINT_KEY); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + + // reset checkpoint to first instant to simulate a random checkpoint for backfill job + // checkpoint will move from 00000 to 00001 for this backfill job + cfgBackfillJob.checkpoint = commitMetadataForFirstInstant.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc); + backfillJob.sync(); + + // check if the checkpoint is carried over + timeline = meta.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + commitMetadataForLastInstant = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + String lastCheckpointAfterParallelBackfill = commitMetadataForLastInstant.getMetadata(CHECKPOINT_KEY); + Assertions.assertEquals(lastCheckpointBeforeParallelBackfill, lastCheckpointAfterParallelBackfill); + } + + private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, int totalRecords, + HoodieDeltaStreamer ingestionJob, HoodieDeltaStreamer.Config cfgIngestionJob, HoodieDeltaStreamer backfillJob, + HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict) throws Exception { + ExecutorService service = Executors.newFixedThreadPool(2); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + String lastSuccessfulCommit = timeline.lastInstant().get().getTimestamp(); + // Condition for parallel ingestion job + Function conditionForRegularIngestion = (r) -> { + if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { + TestHelpers.assertAtleastNDeltaCommitsAfterCommit(3, lastSuccessfulCommit, tableBasePath, dfs); + } else { + TestHelpers.assertAtleastNCompactionCommitsAfterCommit(3, lastSuccessfulCommit, tableBasePath, dfs); + } + TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + return true; + }; + + try { + Future regularIngestionJobFuture = service.submit(() -> { + try { + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, conditionForRegularIngestion); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + }); + Future backfillJobFuture = service.submit(() -> { + try { + backfillJob.sync(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + }); + backfillJobFuture.get(); + regularIngestionJobFuture.get(); + if (expectConflict) { + Assertions.fail("Failed to handle concurrent writes"); + } + } catch (Exception e) { + /** + * Need to perform getMessage().contains since the exception coming + * from {@link org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.DeltaSyncService} gets wrapped many times into RuntimeExceptions. + */ + if (expectConflict && e.getCause().getMessage().contains(ConcurrentModificationException.class.getName())) { + // expected ConcurrentModificationException since ingestion & backfill will have overlapping writes + } else { + throw e; + } + } + } + private void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { try { @@ -682,7 +955,7 @@ private void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer } }); - TestHelpers.waitTillCondition(condition, 240); + TestHelpers.waitTillCondition(condition, dsFuture, 240); ds.shutdownGracefully(); dsFuture.get(); } @@ -1369,4 +1642,13 @@ public Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset rowDataset, + TypedProperties properties) { + return rowDataset; + } + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 946db125289fe..6efd4680c4117 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; @@ -92,6 +93,7 @@ public class UtilitiesTestBase { protected transient SQLContext sqlContext; protected static HiveServer2 hiveServer; protected static HiveTestService hiveTestService; + protected static ZookeeperTestService zookeeperTestService; private static ObjectMapper mapper = new ObjectMapper(); @BeforeAll @@ -105,6 +107,7 @@ public static void initClass() throws Exception { public static void initClass(boolean startHiveService) throws Exception { hdfsTestService = new HdfsTestService(); + zookeeperTestService = new ZookeeperTestService(hdfsTestService.getHadoopConf()); dfsCluster = hdfsTestService.start(true); dfs = dfsCluster.getFileSystem(); dfsBasePath = dfs.getWorkingDirectory().toString(); @@ -114,6 +117,7 @@ public static void initClass(boolean startHiveService) throws Exception { hiveServer = hiveTestService.start(); clearHiveDb(); } + zookeeperTestService.start(); } @AfterAll @@ -127,6 +131,9 @@ public static void cleanupClass() { if (hiveTestService != null) { hiveTestService.stop(); } + if (zookeeperTestService != null) { + zookeeperTestService.stop(); + } } @BeforeEach diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index 84416933f7010..524591dd7bca5 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -126,6 +126,9 @@ protected static Stream fetchNextBatch(TypedProperties props, int } Stream insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .map(AbstractBaseTestSource::toGenericRecord); + if (Boolean.valueOf(props.getOrDefault("hoodie.test.source.generate.inserts", "false").toString())) { + return insertStream; + } return Stream.concat(deleteStream, Stream.concat(updateStream, insertStream)); } diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 1a352f3647cce..6257e97586d65 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -123,6 +123,9 @@ org.apache.hbase:hbase-protocol org.apache.hbase:hbase-server org.apache.htrace:htrace-core + org.apache.curator:curator-framework + org.apache.curator:curator-client + org.apache.curator:curator-recipes commons-codec:commons-codec @@ -347,6 +350,25 @@ hbase-protocol ${hbase.version} + + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} +