From 74241947c123c860a1b0344f25cef316440a70d6 Mon Sep 17 00:00:00 2001 From: n3nash Date: Tue, 16 Mar 2021 16:43:53 -0700 Subject: [PATCH] [HUDI-845] Added locking capability to allow multiple writers (#2374) * [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 | 235 +++++++--- .../transaction/ConcurrentOperation.java | 153 +++++++ .../ConflictResolutionStrategy.java | 64 +++ ...tFileWritesConflictResolutionStrategy.java | 105 +++++ .../transaction/TransactionManager.java | 94 ++++ .../client/transaction/lock/LockManager.java | 135 ++++++ .../lock/ZookeeperBasedLockProvider.java | 168 +++++++ .../client/utils/MetadataConversionUtils.java | 132 ++++++ .../hudi/client/utils/TransactionUtils.java | 128 ++++++ .../hudi/config/HoodieCompactionConfig.java | 1 - .../apache/hudi/config/HoodieLockConfig.java | 191 ++++++++ .../apache/hudi/config/HoodieWriteConfig.java | 99 ++++- .../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 | 107 +++++ ...tFileWritesConflictResolutionStrategy.java | 420 ++++++++++++++++++ .../TestZookeeperBasedLockProvider.java | 116 +++++ .../utils/TestMetadataConversionUtils.java | 213 +++++++++ .../hudi/client/HoodieFlinkWriteClient.java | 6 +- .../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 | 8 +- .../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 +- .../functional/TestHoodieDeltaStreamer.java | 290 +++++++++++- .../testutils/UtilitiesTestBase.java | 7 + .../sources/AbstractBaseTestSource.java | 3 + packaging/hudi-integ-test-bundle/pom.xml | 22 + packaging/hudi-spark-bundle/pom.xml | 24 +- packaging/hudi-utilities-bundle/pom.xml | 22 + pom.xml | 20 + 88 files changed, 4876 insertions(+), 381 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 8829c8580..13f8a74fa 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 676251e06..afca6a62d 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 a3ba00895..9f3b74b0c 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 com.codahale.metrics.Timer; 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,20 @@ import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; 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.client.utils.TransactionUtils; 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 +107,8 @@ public abstract class AbstractHoodieWriteClient>> lastCompletedTxnAndMetadata = Option.empty(); /** * Create a write client, with new hudi index. @@ -124,6 +132,7 @@ public abstract class AbstractHoodieWriteClient createIndex(HoodieWriteConfig writeConfig); @@ -163,26 +172,28 @@ public abstract class AbstractHoodieWriteClient 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)), + lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); 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); + 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) { @@ -193,6 +204,16 @@ public abstract class AbstractHoodieWriteClient 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 +231,11 @@ public abstract class AbstractHoodieWriteClient> 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 +388,20 @@ public abstract class AbstractHoodieWriteClient table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { - // 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"); - } - - // 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"); - } - // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); @@ -419,6 +438,28 @@ public abstract class AbstractHoodieWriteClient 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 -> { @@ -587,6 +628,20 @@ public abstract class AbstractHoodieWriteClient scheduleCompaction(Option> extraMetadata) throws HoodieIOException { @@ -673,15 +727,11 @@ public abstract class AbstractHoodieWriteClient> 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(); } /** @@ -723,14 +773,14 @@ public abstract class AbstractHoodieWriteClient 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 @@ public abstract class AbstractHoodieWriteClient 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 abstract class AbstractHoodieWriteClient 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 @@ public abstract class AbstractHoodieWriteClient inlineCompact(Option> extraMetadata) { Option compactionInstantTimeOpt = scheduleCompaction(extraMetadata); - compactionInstantTimeOpt.ifPresent(compactionInstantTime -> { + 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 { @@ -816,25 +869,93 @@ public abstract class AbstractHoodieWriteClient> 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(); + } + + /** + * Schedules a new cleaning instant. + * @param extraMetadata Extra Metadata to be stored + */ + protected Option scheduleCleaning(Option> extraMetadata) throws HoodieIOException { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleCleaningAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + } + + /** + * 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 { + 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 */ public abstract HoodieWriteMetadata 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: + LOG.info("Scheduling clustering at instant time :" + instantTime); + Option clusteringPlan = createTable(config, hadoopConf) + .scheduleClustering(context, instantTime, extraMetadata); + return clusteringPlan.isPresent() ? Option.of(instantTime) : Option.empty(); + case COMPACT: + 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: + 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); + } + } + /** * Executes a clustering plan on a table, serially before or after an insert/upsert action. */ @@ -923,12 +1044,12 @@ public abstract class AbstractHoodieWriteClient 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 000000000..d1e988adb --- /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 000000000..938a40684 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -0,0 +1,105 @@ +/* + * 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. + 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 000000000..a6753aaa3 --- /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 000000000..4b8004c03 --- /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 000000000..60336c53e --- /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 000000000..b46f3d866 --- /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.getActiveTimeline().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.getActiveTimeline().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.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); + archivedMetaWrapper.setActionType(ActionType.rollback.name()); + break; + } + case HoodieTimeline.SAVEPOINT_ACTION: { + archivedMetaWrapper.setHoodieSavePointMetadata(TimelineMetadataUtils.deserializeAvroMetadata( + metaClient.getActiveTimeline().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 000000000..80a412010 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java @@ -0,0 +1,128 @@ +/* + * 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 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; +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"); + // 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/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index e05195b9d..43af00a13 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 000000000..1f0328e2f --- /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 4e493e443..944cd0230 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.MetricsReporterType; 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.Properties; 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,15 @@ 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(); + + // 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 @@ -427,7 +440,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 +472,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 +482,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public boolean isClusteringEnabled() { // TODO: future support async clustering - return isInlineClustering() || isAsyncClusteringEnabled(); + return inlineClusteringEnabled() || isAsyncClusteringEnabled(); } public int getInlineClusterMaxCommits() { @@ -939,10 +952,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 +991,47 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 String getWriteMetaKeyPrefixes() { + return props.getProperty(WRITE_META_KEY_PREFIXES_PROP); + } + public static class Builder { protected final Properties props = new Properties(); @@ -998,6 +1048,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 +1192,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 +1311,16 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { return this; } + public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) { + props.setProperty(WRITE_CONCURRENCY_MODE_PROP, concurrencyMode.value()); + 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; @@ -1314,7 +1381,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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); + 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()); @@ -1336,6 +1406,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { 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 +1421,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { // 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 dbd678f2b..dfd4c4c04 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.HoodiePartitionMetadata; 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 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta // 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 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta .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 4e72f1871..fd5321a97 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.FileSystem; 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; @@ -393,6 +394,19 @@ public abstract class HoodieTable implem */ 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 8efd3a226..09df62c91 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.HoodieActiveTimeline; 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.HoodieCommitException; 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 @@ public class HoodieTimelineArchiveLog { 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 5b3782007..acc3cdc67 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 @@ package org.apache.hudi.table.action.clean; 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 org.apache.log4j.Logger; 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 abstract class BaseCleanActionExecutor 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 @@ public abstract class BaseCleanActionExecutor 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 @@ public abstract class BaseCleanActionExecutor 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 abstract class BaseCleanActionExecutor { 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 000000000..fc0c000a6 --- /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 4a8ddf5c5..16638fd95 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,7 +31,9 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; @@ -58,6 +62,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 +72,9 @@ public abstract class BaseCommitActionExecutor execute(I inputRecords); @@ -117,12 +126,24 @@ public abstract class BaseCommitActionExecutor> extraMetadata, HoodieWriteMetadata result) { + this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime)), + lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); + try { + TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), + result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner()); + commit(extraMetadata, result); + } finally { + this.txnManager.endTransaction(); + } + } + protected abstract void commit(Option> extraMetadata, HoodieWriteMetadata result); /** @@ -140,6 +161,10 @@ public abstract class BaseCommitActionExecutor 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 7476a1630..67bc1c209 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 @@ public abstract class AbstractUpgradeDowngrade { 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 000000000..ed0b83b54 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.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.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.fs.FSUtils; +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) { + this.lockConfiguration = lockConfiguration; + this.lockPath = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP); + this.fs = FSUtils.getFs(this.lockPath, configuration); + } + + 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.delete(new Path(lockPath + "/" + LOCK_NAME), true); + } catch (IOException e) { + throw new HoodieLockException("Unable to release lock", e); + } + } + + @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 000000000..9483d61fb --- /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 000000000..5e36399a6 --- /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 000000000..d8a51708f --- /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 5d86d0a87..21a396c78 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 @@ -134,7 +134,7 @@ public class HoodieFlinkWriteClient extends 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 +156,7 @@ public class HoodieFlinkWriteClient extends 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 +193,7 @@ public class HoodieFlinkWriteClient extends 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-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 e7f1d592e..167b3766e 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.io.HoodieWriteHandle; 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 class HoodieFlinkCopyOnWriteTable extends 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 000000000..75da54e31 --- /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 e0bbc25ef..5df46a65b 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 @@ public abstract class BaseFlinkCommitActionExecutor extends 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 class HoodieJavaWriteClient extends 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 class HoodieJavaWriteClient extends 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 class HoodieJavaWriteClient extends 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 class HoodieJavaWriteClient extends 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 157e11a55..06e66a1a0 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.exception.HoodieNotSupportedException; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.JavaCleanActionExecutor; +import org.apache.hudi.table.action.clean.JavaScheduleCleanActionExecutor; import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaBulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor; @@ -175,6 +177,11 @@ public class HoodieJavaCopyOnWriteTable extends H 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 000000000..05d19a63e --- /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 a4a6a4f92..a2ea1c9c0 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 @@ public abstract class BaseJavaCommitActionExecutor 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 091f1dafa..8a0eee59a 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 com.codahale.metrics.Timer; 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.HoodieSparkTable; 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends 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 class SparkRDDWriteClient extends @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 @@ public class SparkRDDWriteClient extends @Override protected 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 @@ public class SparkRDDWriteClient extends 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 @@ public class SparkRDDWriteClient extends @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 class SparkRDDWriteClient extends 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 @@ public class SparkRDDWriteClient extends @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 class SparkRDDWriteClient extends } } + @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 12867e636..98063f6cf 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.HoodieWriteMetadata; 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 class HoodieSparkCopyOnWriteTable extends 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 1aca28b6d..4f204eabd 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.config.HoodieWriteConfig; 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 @@ public class SparkBootstrapCommitActionExecutor 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 @@ public class SparkBootstrapCommitActionExecutor // 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 000000000..f5529a8b7 --- /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 a99b00162..aa199c2b5 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.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.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 @@ public abstract class BaseSparkCommitActionExecutor> 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 000000000..a56789bd1 --- /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 41ee4b208..5c2b61521 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.HoodieRequestedReplaceMetadata; 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.HoodieKey; 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.exception.HoodieUpsertException; 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.HashMap; 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.Executors; 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 042013d3a..5b1888101 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.HoodieActionInstant; 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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { 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 52308593f..f34f567b1 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 class TestHoodieBackedMetadata extends HoodieClientTestHarness { 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 fd578bd8b..83bb68429 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 @@ public class TestCleaner extends HoodieClientTestBase { }); }); 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 c054bc460..3e03c0536 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 class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { .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 class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { 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 72b277128..951e17ca3 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.testutils.minicluster.HdfsTestService; 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 000000000..d9be6a647 --- /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 000000000..7d8e52738 --- /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 000000000..e40d5e5a9 --- /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 000000000..ecc18a7df --- /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 000000000..90444a3d6 --- /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 000000000..9fe66ca0a --- /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 f23715636..41cfc8abc 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 enum WriteOperationType { 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 fa494931a..c7d61a732 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 class HoodieDefaultTimeline implements HoodieTimeline { 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 b48543c3d..1e366147a 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 9c53cec4e..1bf97c68e 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 class ClusteringUtils { .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 1081d9c1f..ea36f6739 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.hudi.exception.HoodieException; 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 @@ public class CommitUtils { + "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 000000000..cd1ff7f0e --- /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 000000000..f0f6dcbf0 --- /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 5f03ef015..5474b184b 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 @@ package org.apache.hudi.common.testutils; 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 java.util.Map; 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 class FileCreateUtils { 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 class FileCreateUtils { 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 class FileCreateUtils { 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 46ef14f20..182a8d409 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.HoodieCleanMetadata; 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.createInflightCom 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.createRequestedCo 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 class HoodieTestTable { 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 class HoodieTestTable { 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 class HoodieTestTable { 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 c27877008..e5c228f40 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 class ZookeeperTestService { 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 @@ -232,4 +232,8 @@ public class ZookeeperTestService { } 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 98535e952..c55e34acf 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 @@ public class TestCommitUtils { 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 d93c663e8..bf0037377 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 da7953f98..42bbfbfa6 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.DataSourceUtils; 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.DagNode; 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 class HoodieTestSuiteJob { 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 class HoodieTestSuiteJob { 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 class HoodieTestSuiteJob { + "(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 c3294dede..a30312f7f 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.model.WriteOperationType; 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.DagNode; 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 @@ import java.util.Set; * 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 @@ public class HoodieTestSuiteWriter { 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 class HoodieTestSuiteWriter { 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 650ab1eea..27760f711 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 class WriterContext { 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 000000000..afd8a0759 --- /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 4bd096ae0..054870f1f 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 class DFSDeltaWriterAdapter implements DeltaWriterAdapter 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 000000000..cb25a6d2e --- /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 000000000..1cc33eda5 --- /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 6232b1dfc..ec470e023 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 class TestHoodieTestSuiteJob extends UtilitiesTestBase { 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 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase { 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 class TestHoodieTestSuiteJob extends UtilitiesTestBase { //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 000000000..16023f69e --- /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 d2c948502..171513138 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 4adcf242d..77a14ac50 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 @@ -505,8 +505,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 000000000..4a96c7890 --- /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 000000000..51f7814c3 --- /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 2090b65de..ca7aa7a02 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 class HiveTestService { 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 5feca25b1..80a681db4 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 class HiveTestUtil { 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 class HiveTestUtil { 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 17eee8ee5..336639c98 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 @@ public class DeltaSync implements Serializable { } // 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/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 7522c2d29..8f3e04521 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.sql.types.StructField; 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.function.Function; 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 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { 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 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { 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 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { 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 class TestHoodieDeltaStreamer extends UtilitiesTestBase { 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 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { }); } + 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 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { } }); - TestHelpers.waitTillCondition(condition, 240); + TestHelpers.waitTillCondition(condition, dsFuture, 240); ds.shutdownGracefully(); dsFuture.get(); } @@ -1369,4 +1642,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { return sparkSession.createDataFrame(jsc.emptyRDD(), rowDataset.schema()); } } + + public static class TestIdentityTransformer implements Transformer { + + @Override + public 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 946db1252..6efd4680c 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.HoodieTestDataGenerator; 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 class UtilitiesTestBase { 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 class UtilitiesTestBase { hiveServer = hiveTestService.start(); clearHiveDb(); } + zookeeperTestService.start(); } @AfterAll @@ -127,6 +131,9 @@ public class UtilitiesTestBase { 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 84416933f..524591dd7 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 @@ public abstract class AbstractBaseTestSource extends AvroSource { } 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-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index db2635af2..c24021568 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 ba91f5498..e20123e5c 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/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 1a352f364..6257e9758 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} + diff --git a/pom.xml b/pom.xml index 569ce7b50..ad726c0d7 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