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

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

+ *

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

+ */ +public class HoodieLockException extends HoodieException { + + public HoodieLockException(String msg) { + super(msg); + } + + public HoodieLockException(Throwable e) { + super(e); + } + + public HoodieLockException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java new file mode 100644 index 0000000000000..f0f6dcbf0ab17 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieWriteConflictException.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +/** + *

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

+ *

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

+ */ +public class HoodieWriteConflictException extends HoodieException { + + public HoodieWriteConflictException(String msg) { + super(msg); + } + + public HoodieWriteConflictException(Throwable e) { + super(e); + } + + public HoodieWriteConflictException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 5f03ef015faf0..5474b184bad54 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -22,14 +22,15 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.exception.HoodieException; @@ -47,6 +48,8 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanerPlan; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRequestedReplaceMetadata; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata; public class FileCreateUtils { @@ -117,6 +120,10 @@ public static void createCommit(String basePath, String instantTime) throws IOEx createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } + public static void createCommit(String basePath, String instantTime, HoodieCommitMetadata metadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + } + public static void createCommit(String basePath, String instantTime, FileSystem fs) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION, fs); } @@ -149,9 +156,8 @@ public static void createReplaceCommit(String basePath, String instantTime, Hood createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); } - public static void createRequestedReplaceCommit(String basePath, String instantTime, HoodieRequestedReplaceMetadata requestedMetadata) throws IOException { - createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION, - TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedMetadata).get()); + public static void createRequestedReplaceCommit(String basePath, String instantTime, HoodieRequestedReplaceMetadata requestedReplaceMetadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION, serializeRequestedReplaceMetadata(requestedReplaceMetadata).get()); } public static void createInflightReplaceCommit(String basePath, String instantTime) throws IOException { @@ -170,6 +176,14 @@ public static void createInflightCleanFile(String basePath, String instantTime, createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION, serializeCleanerPlan(cleanerPlan).get()); } + public static void createInflightRollbackFile(String basePath, String instantTime) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); + } + + public static void createRollbackFile(String basePath, String instantTime, HoodieRollbackMetadata rollbackMetadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION, serializeRollbackMetadata(rollbackMetadata).get()); + } + private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); Files.createDirectories(parentPath); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 46ef14f2077a0..182a8d409ab24 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -23,7 +23,9 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.IOType; @@ -65,6 +67,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightReplaceCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCleanFile; @@ -72,6 +75,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; public class HoodieTestTable { @@ -155,6 +159,15 @@ public HoodieTestTable addCommit(String instantTime) throws Exception { return this; } + public HoodieTestTable addCommit(String instantTime, HoodieCommitMetadata metadata) throws Exception { + createRequestedCommit(basePath, instantTime); + createInflightCommit(basePath, instantTime); + createCommit(basePath, instantTime, metadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addDeltaCommit(String instantTime) throws Exception { createRequestedDeltaCommit(basePath, instantTime); createInflightDeltaCommit(basePath, instantTime); @@ -173,6 +186,13 @@ public HoodieTestTable addReplaceCommit(String instantTime, HoodieRequestedRepla return this; } + public HoodieTestTable addRequestedReplace(String instantTime, HoodieRequestedReplaceMetadata requestedReplaceMetadata) throws Exception { + createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addInflightClean(String instantTime, HoodieCleanerPlan cleanerPlan) throws IOException { createRequestedCleanFile(basePath, instantTime, cleanerPlan); createInflightCleanFile(basePath, instantTime, cleanerPlan); @@ -190,6 +210,14 @@ public HoodieTestTable addClean(String instantTime, HoodieCleanerPlan cleanerPla return this; } + public HoodieTestTable addRollback(String instantTime, HoodieRollbackMetadata rollbackMetadata) throws IOException { + createInflightRollbackFile(basePath, instantTime); + createRollbackFile(basePath, instantTime, rollbackMetadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addRequestedCompaction(String instantTime) throws IOException { createRequestedCompaction(basePath, instantTime); currentInstantTime = instantTime; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java index c278770081904..e5c228f40432b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/minicluster/ZookeeperTestService.java @@ -124,14 +124,14 @@ public ZooKeeperServer start() throws IOException, InterruptedException { return zooKeeperServer; } - public void stop() throws IOException { + public void stop() throws RuntimeException { if (!started) { return; } standaloneServerFactory.shutdown(); if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) { - throw new IOException("Waiting for shutdown of standalone server"); + throw new RuntimeException("Waiting for shutdown of standalone server"); } // clear everything @@ -232,4 +232,8 @@ private static boolean waitForServerUp(String hostname, int port, long timeout) } return false; } + + public String connectString() { + return bindIP + ":" + clientPort; + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java index 98535e95238e6..c55e34acfa877 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java @@ -93,4 +93,4 @@ private HoodieWriteStat createWriteStat(String partition, String fileId) { writeStat1.setFileId(fileId); return writeStat1; } -} +} \ No newline at end of file diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index d93c663e8e44a..bf00373774048 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -81,6 +81,10 @@ org.eclipse.jetty * + + org.apache.curator + * + @@ -89,11 +93,23 @@ org.apache.hudi hudi-spark_${scala.binary.version} ${project.version} + + + org.apache.curator + * + + org.apache.hudi hudi-spark-common ${project.version} + + + org.apache.curator + * + + org.apache.hudi @@ -117,9 +133,29 @@ com.databricks spark-avro_${scala.binary.version} + + org.apache.curator + * + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + org.apache.curator + curator-client + ${zk-curator.version} + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + log4j @@ -141,7 +177,6 @@ ${project.version} tests test-jar - test org.apache.hudi @@ -213,6 +248,13 @@ test-jar + + org.apache.curator + curator-test + ${zk-curator.version} + test + + com.fasterxml.jackson.dataformat diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index da7953f982e8b..42bbfbfa6a2a0 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -23,13 +23,13 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.dag.DagUtils; import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator; @@ -38,6 +38,8 @@ import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode; import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler; import org.apache.hudi.integ.testsuite.dag.scheduler.SaferSchemaDagScheduler; +import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider; +import org.apache.hudi.integ.testsuite.helpers.ZookeeperServiceProvider; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; import org.apache.hudi.keygen.BuiltinKeyGenerator; @@ -182,9 +184,9 @@ public void runTestSuite() { long startTime = System.currentTimeMillis(); WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); writerContext.initContext(jsc); + startOtherServicesIfNeeded(writerContext); if (this.cfg.saferSchemaEvolution) { int numRollbacks = 2; // rollback most recent upsert/insert, by default. - // if root is RollbackNode, get num_rollbacks List root = workflowDag.getNodeList(); if (!root.isEmpty() && root.get(0) instanceof RollbackNode) { @@ -203,8 +205,31 @@ public void runTestSuite() { log.error("Failed to run Test Suite ", e); throw new HoodieException("Failed to run Test Suite ", e); } finally { + stopQuietly(); + } + } + + private void stopQuietly() { + try { sparkSession.stop(); jsc.stop(); + } catch (Exception e) { + log.error("Unable to stop spark session", e); + } + } + + private void startOtherServicesIfNeeded(WriterContext writerContext) throws Exception { + if (cfg.startHiveMetastore) { + HiveServiceProvider hiveServiceProvider = new HiveServiceProvider( + Config.newBuilder().withHiveLocal(true).build()); + hiveServiceProvider.startLocalHiveServiceIfNeeded(writerContext.getHoodieTestSuiteWriter().getConfiguration()); + hiveServiceProvider.syncToLocalHiveIfNeeded(writerContext.getHoodieTestSuiteWriter()); + } + + if (cfg.startZookeeper) { + ZookeeperServiceProvider zookeeperServiceProvider = new ZookeeperServiceProvider(Config.newBuilder().withHiveLocal(true).build(), + writerContext.getHoodieTestSuiteWriter().getConfiguration()); + zookeeperServiceProvider.startLocalZookeeperIfNeeded(); } } @@ -262,5 +287,11 @@ public static class HoodieTestSuiteConfig extends HoodieDeltaStreamer.Config { + "(If not provided, assumed to be false.)", required = false) public Boolean saferSchemaEvolution = false; + + @Parameter(names = {"--start-zookeeper"}, description = "Start Zookeeper instance to use for optimistic lock ") + public Boolean startZookeeper = false; + + @Parameter(names = {"--start-hive-metastore"}, description = "Start Hive Metastore to use for optimistic lock ") + public Boolean startHiveMetastore = false; } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index c3294dede6d09..a30312f7fa525 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -18,6 +18,9 @@ package org.apache.hudi.integ.testsuite; +import java.io.Serializable; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; @@ -28,7 +31,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; @@ -40,12 +42,12 @@ import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode; import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode; import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; -import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +62,7 @@ * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are * {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}. */ -public class HoodieTestSuiteWriter { +public class HoodieTestSuiteWriter implements Serializable { private static Logger log = LoggerFactory.getLogger(HoodieTestSuiteWriter.class); @@ -121,6 +123,14 @@ private boolean allowWriteClientAccess(DagNode dagNode) { return false; } + public RDD getNextBatch() throws Exception { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + JavaRDD inputRDD = nextBatch.getRight().getRight(); + return inputRDD.map(r -> (GenericRecord) r.getData() + .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); + } + public Pair>> fetchSource() throws Exception { return this.deltaStreamerWrapper.fetchSource(); } @@ -253,4 +263,12 @@ public JavaSparkContext getSparkContext() { public Option getLastCheckpoint() { return lastCheckpoint; } + + public Properties getProps() { + return props; + } + + public String getSchema() { + return schema; + } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index 650ab1eeaf4c2..27760f7116ecd 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -109,4 +109,8 @@ public TypedProperties getProps() { public String toString() { return this.hoodieTestSuiteWriter.toString() + "\n" + this.deltaGenerator.toString() + "\n"; } + + public SparkSession getSparkSession() { + return sparkSession; + } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.java new file mode 100644 index 0000000000000..afd8a07590587 --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/ZookeeperServiceProvider.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.helpers; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; + +public class ZookeeperServiceProvider { + + Configuration configuration; + private CuratorFramework client; + private Config config; + + public ZookeeperServiceProvider(Config config, Configuration configuration) { + this.configuration = configuration; + this.config = config; + } + + public void startLocalZookeeperIfNeeded() throws Exception { + if (config.isHiveLocal()) { + ZookeeperTestService zookeeperTestService = new ZookeeperTestService(configuration); + zookeeperTestService.start(); + } + } + + public void stopLocalZookeeperIfNeeded() throws Exception { + if (config.isHiveLocal()) { + ZookeeperTestService zookeeperTestService = new ZookeeperTestService(configuration); + zookeeperTestService.stop(); + } + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java index 4bd096ae0cbde..054870f1fe653 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java @@ -49,7 +49,6 @@ public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputWriter) { public List write(Iterator input) throws IOException { while (input.hasNext()) { GenericRecord next = input.next(); - next.put(SchemaUtils.SOURCE_ORDERING_FIELD, preCombineFieldVal); if (this.deltaInputWriter.canWrite()) { this.deltaInputWriter.writeData(next); } else { diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala new file mode 100644 index 0000000000000..cb25a6d2e1698 --- /dev/null +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes + +import org.apache.hudi.client.WriteStatus +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config +import org.apache.hudi.integ.testsuite.dag.ExecutionContext +import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SaveMode + +import scala.collection.JavaConverters._ + +/** + * Spark datasource based insert node + * @param config1 + */ +class SparkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] { + + config = config1 + + /** + * Execute the {@link DagNode}. + * + * @param context The context needed for an execution of a node. + * @param curItrCount iteration count for executing the node. + * @throws Exception Thrown if the execution failed. + */ + override def execute(context: ExecutionContext, curItrCount: Int): Unit = { + if (!config.isDisableGenerate) { + println("Generating input data for node {}", this.getName) + context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count() + } + val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch, + context.getWriterContext.getHoodieTestSuiteWriter.getSchema, + context.getWriterContext.getSparkSession) + inputDF.write.format("hudi") + .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key") + .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse("")) + .option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .mode(SaveMode.Overwrite) + .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath) + } +} diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala new file mode 100644 index 0000000000000..1cc33eda5c5ce --- /dev/null +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes + +import org.apache.hudi.client.WriteStatus +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config +import org.apache.hudi.integ.testsuite.dag.ExecutionContext +import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SaveMode + +import scala.collection.JavaConverters._ + +/** + * Spark datasource based upsert node + * @param config1 + */ +class SparkUpsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] { + + config = config1 + + /** + * Execute the {@link DagNode}. + * + * @param context The context needed for an execution of a node. + * @param curItrCount iteration count for executing the node. + * @throws Exception Thrown if the execution failed. + */ + override def execute(context: ExecutionContext, curItrCount: Int): Unit = { + if (!config.isDisableGenerate) { + println("Generating input data for node {}", this.getName) + context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count() + } + val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch, + context.getWriterContext.getHoodieTestSuiteWriter.getSchema, + context.getWriterContext.getSparkSession) + inputDF.write.format("hudi") + .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key") + .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse("")) + .option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .mode(SaveMode.Append) + .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath) + } +} diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index 6232b1dfc2f89..ec470e0231712 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -69,6 +69,8 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase { private static final String MOR_DAG_FILE_NAME = "unit-test-mor-dag.yaml"; private static final String MOR_DAG_SOURCE_PATH = "/hudi-integ-test/src/test/resources/" + MOR_DAG_FILE_NAME; + private static final String COW_DAG_FILE_NAME_SPARK_DATASOURCE_NODES = "unit-test-cow-dag-spark-datasource.yaml"; + private static final String COW_DAG_SPARK_DATASOURCE_NODES_RELATIVE_PATH = "/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml"; public static Stream configParams() { Object[][] data = @@ -92,39 +94,15 @@ public static void initClass() throws Exception { UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.." + MOR_DAG_SOURCE_PATH, dfs, dfsBasePath + "/" + MOR_DAG_FILE_NAME); - TypedProperties props = new TypedProperties(); - props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); - props.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp"); - props.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", "UNIX_TIMESTAMP"); - props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd"); - props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); - props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc"); - props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/input"); - props.setProperty("hoodie.datasource.hive_sync.assume_date_partitioning", "true"); - props.setProperty("hoodie.datasource.hive_sync.skip_ro_suffix", "true"); - props.setProperty("hoodie.datasource.write.keytranslator.class", "org.apache.hudi" - + ".DayBasedPartitionPathKeyTranslator"); - props.setProperty("hoodie.compact.inline.max.delta.commits", "3"); - props.setProperty("hoodie.parquet.max.file.size", "1024000"); - props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); - props.setProperty("hoodie.index.type", HoodieIndex.IndexType.GLOBAL_SIMPLE.name()); - props.setProperty("hoodie.global.simple.index.parallelism", "2"); - // Reduce shuffle parallelism, spark hangs when numPartitions >> numRecords to process - props.setProperty("hoodie.insert.shuffle.parallelism", "10"); - props.setProperty("hoodie.upsert.shuffle.parallelism", "10"); - props.setProperty("hoodie.bulkinsert.shuffle.parallelism", "10"); - props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); - // Make path selection test suite specific - props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); - // Hive Configs - props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1"); - props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "table1"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr"); - props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), TimestampBasedKeyGenerator.class.getName()); + TypedProperties props = getProperties(); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source" + ".properties"); + UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.." + + COW_DAG_SPARK_DATASOURCE_NODES_RELATIVE_PATH, dfs, dfsBasePath + "/" + COW_DAG_FILE_NAME_SPARK_DATASOURCE_NODES); + UtilitiesTestBase.Helpers.savePropsToDFS(getProperties(), dfs, dfsBasePath + "/test-source" + + ".properties"); + // Properties used for the delta-streamer which incrementally pulls from upstream DFS Avro source and // writes to downstream hudi table TypedProperties downstreamProps = new TypedProperties(); @@ -161,6 +139,50 @@ private void cleanDFSDirs() throws Exception { dfs.delete(new Path(dfsBasePath + "/result"), true); } + private static TypedProperties getProperties() { + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + props.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp"); + props.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", "UNIX_TIMESTAMP"); + props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd"); + props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); + props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc"); + props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/input"); + props.setProperty("hoodie.datasource.hive_sync.assume_date_partitioning", "true"); + props.setProperty("hoodie.datasource.hive_sync.skip_ro_suffix", "true"); + props.setProperty("hoodie.datasource.write.keytranslator.class", "org.apache.hudi" + + ".DayBasedPartitionPathKeyTranslator"); + props.setProperty("hoodie.compact.inline.max.delta.commits", "3"); + props.setProperty("hoodie.parquet.max.file.size", "1024000"); + props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); + props.setProperty("hoodie.index.type", HoodieIndex.IndexType.GLOBAL_SIMPLE.name()); + props.setProperty("hoodie.global.simple.index.parallelism", "2"); + // Reduce shuffle parallelism, spark hangs when numPartitions >> numRecords to process + props.setProperty("hoodie.insert.shuffle.parallelism", "10"); + props.setProperty("hoodie.upsert.shuffle.parallelism", "10"); + props.setProperty("hoodie.bulkinsert.shuffle.parallelism", "10"); + props.setProperty("hoodie.compact.inline.max.delta.commits", "0"); + // Make path selection test suite specific + props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); + // Hive Configs + props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1"); + props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "table1"); + props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr"); + props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), TimestampBasedKeyGenerator.class.getName()); + + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); + props.setProperty("hoodie.writer.lock.hivemetastore.database", "testdb1"); + props.setProperty("hoodie.writer.lock.hivemetastore.table", "table1"); + props.setProperty("hoodie.writer.lock.zookeeper.url", "127.0.0.1"); + props.setProperty("hoodie.writer.lock.zookeeper.port", "2828"); + props.setProperty("hoodie.writer.lock.wait_time_ms", "1200000"); + props.setProperty("hoodie.writer.lock.num_retries", "10"); + props.setProperty("hoodie.writer.lock.lock_key", "test_table"); + props.setProperty("hoodie.writer.lock.zookeeper.zk_base_path", "/test"); + return props; + } + // Tests in this class add to the test build time significantly. Since this is a Integration Test (end to end), we // would like to run this as a nightly build which is a TODO. // TODO : Clean up input / result paths after each test @@ -226,6 +248,27 @@ public void testMORFullDagFromYaml() throws Exception { //assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7); } + @Test + public void testSparkDataSourceNodesDagWithLock() throws Exception { + boolean useDeltaStreamer = false; + this.cleanDFSDirs(); + + TypedProperties props = getProperties(); + props.setProperty("hoodie.write.concurrency.mode", "optimistic_concurrency_control"); + props.setProperty("hoodie.failed.writes.cleaner.policy", "LAZY"); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source" + + ".properties"); + String inputBasePath = dfsBasePath + "/input"; + String outputBasePath = dfsBasePath + "/result"; + HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, HoodieTableType + .COPY_ON_WRITE.name()); + cfg.workloadYamlPath = dfsBasePath + "/" + COW_DAG_FILE_NAME_SPARK_DATASOURCE_NODES; + HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc); + hoodieTestSuiteJob.runTestSuite(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(cfg.targetBasePath).build(); + assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 3); + } + protected HoodieTestSuiteConfig makeConfig(String inputBasePath, String outputBasePath, boolean useDeltaStream, String tableType) { HoodieTestSuiteConfig cfg = new HoodieTestSuiteConfig(); diff --git a/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml b/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml new file mode 100644 index 0000000000000..16023f69ed5e8 --- /dev/null +++ b/hudi-integ-test/src/test/resources/unit-test-cow-dag-spark-datasource.yaml @@ -0,0 +1,43 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +dag_name: unit-test-cow-dag-spark-datasource +dag_rounds: 1 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 100 + type: SparkInsertNode + deps: none + first_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: SparkUpsertNode + deps: first_insert + second_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: SparkUpsertNode + deps: first_upsert \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index d2c9485020c53..171513138c479 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -203,6 +203,12 @@ org.apache.hudi hudi-spark-common ${project.version} + + + org.apache.curator + * + + org.apache.hudi @@ -379,6 +385,24 @@ + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + org.apache.hudi diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 1f1dc4d4233bf..8c1716fb3c53b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -497,8 +497,8 @@ private[hudi] object HoodieSparkSqlWriter { private def isAsyncCompactionEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]], tableConfig: HoodieTableConfig, parameters: Map[String, String], configuration: Configuration) : Boolean = { - log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}") - if (asyncCompactionTriggerFnDefined && !client.getConfig.isInlineCompaction + log.info(s"Config.inlineCompactionEnabled ? ${client.getConfig.inlineCompactionEnabled}") + if (asyncCompactionTriggerFnDefined && !client.getConfig.inlineCompactionEnabled && parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY).exists(r => r.toBoolean)) { tableConfig.getTableType == HoodieTableType.MERGE_ON_READ } else { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java new file mode 100644 index 0000000000000..4a96c7890c51e --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreLockProvider.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hive; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.thrift.TException; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.lock.LockState.ACQUIRING; +import static org.apache.hudi.common.lock.LockState.ALREADY_ACQUIRED; +import static org.apache.hudi.common.lock.LockState.FAILED_TO_ACQUIRE; +import static org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE; +import static org.apache.hudi.common.lock.LockState.RELEASED; +import static org.apache.hudi.common.lock.LockState.RELEASING; + +/** + * A hivemetastore based lock. Default HiveMetastore Lock Manager uses zookeeper to provide locks, read here + * {@link /cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-Locking} + * This {@link LockProvider} implementation allows to lock table operations + * using hive metastore APIs. Users need to have a HiveMetastore & Zookeeper cluster deployed to be able to use this lock. + * + */ +public class HiveMetastoreLockProvider implements LockProvider { + + private static final Logger LOG = LogManager.getLogger(HiveMetastoreLockProvider.class); + + private final String databaseName; + private final String tableName; + private IMetaStoreClient hiveClient; + private volatile LockResponse lock = null; + protected LockConfiguration lockConfiguration; + ExecutorService executor = Executors.newSingleThreadExecutor(); + + public HiveMetastoreLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + this(lockConfiguration); + try { + HiveConf hiveConf = new HiveConf(); + setHiveLockConfs(hiveConf); + hiveConf.addResource(conf); + this.hiveClient = Hive.get(hiveConf).getMSC(); + } catch (MetaException | HiveException e) { + throw new HoodieLockException("Failed to create HiveMetaStoreClient", e); + } + } + + public HiveMetastoreLockProvider(final LockConfiguration lockConfiguration, final IMetaStoreClient metaStoreClient) { + this(lockConfiguration); + this.hiveClient = metaStoreClient; + } + + HiveMetastoreLockProvider(final LockConfiguration lockConfiguration) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.databaseName = this.lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP); + this.tableName = this.lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP); + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + } catch (ExecutionException | InterruptedException | TimeoutException | TException e) { + throw new HoodieLockException(generateLogStatement(FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } + return this.lock != null && this.lock.getState() == LockState.ACQUIRED; + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(RELEASING, generateLogSuffixString())); + LockResponse lockResponseLocal = lock; + if (lockResponseLocal == null) { + return; + } + lock = null; + hiveClient.unlock(lockResponseLocal.getLockid()); + LOG.info(generateLogStatement(RELEASED, generateLogSuffixString())); + } catch (TException e) { + throw new HoodieLockException(generateLogStatement(FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + public void acquireLock(long time, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException, TException { + ValidationUtils.checkArgument(this.lock == null, ALREADY_ACQUIRED.name()); + final LockComponent lockComponent = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, this.databaseName); + lockComponent.setTablename(tableName); + acquireLockInternal(time, unit, lockComponent); + } + + // NOTE: HiveMetastoreClient does not implement AutoCloseable. Additionally, we cannot call close() after unlock() + // because if there are multiple operations started from the same WriteClient (via multiple threads), closing the + // hive client causes all other threads who may have already initiated the tryLock() to fail since the + // HiveMetastoreClient is shared. + @Override + public void close() { + try { + if (lock != null) { + hiveClient.unlock(lock.getLockid()); + } + hiveClient.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + public IMetaStoreClient getHiveClient() { + return hiveClient; + } + + @Override + public LockResponse getLock() { + return this.lock; + } + + // This API is exposed for tests and not intended to be used elsewhere + public boolean acquireLock(long time, TimeUnit unit, final LockComponent component) + throws InterruptedException, ExecutionException, TimeoutException, TException { + ValidationUtils.checkArgument(this.lock == null, ALREADY_ACQUIRED.name()); + acquireLockInternal(time, unit, component); + return this.lock != null && this.lock.getState() == LockState.ACQUIRED; + } + + private void acquireLockInternal(long time, TimeUnit unit, LockComponent lockComponent) + throws InterruptedException, ExecutionException, TimeoutException, TException { + LockRequest lockRequest = null; + try { + final LockRequestBuilder builder = new LockRequestBuilder("hudi-lock"); + lockRequest = builder.addLockComponent(lockComponent).setUser(System.getProperty("user.name")).build(); + lockRequest.setUserIsSet(true); + final LockRequest lockRequestFinal = lockRequest; + this.lock = executor.submit(() -> hiveClient.lock(lockRequestFinal)) + .get(time, unit); + } catch (InterruptedException | TimeoutException e) { + if (this.lock != null && this.lock.getState() == LockState.ACQUIRED) { + return; + } else if (lockRequest != null) { + LockResponse lockResponse = this.hiveClient.checkLock(lockRequest.getTxnid()); + if (lockResponse.getState() == LockState.ACQUIRED) { + this.lock = lockResponse; + return; + } else { + throw e; + } + } else { + throw e; + } + } + } + + private void checkRequiredProps(final LockConfiguration lockConfiguration) { + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_DATABASE_NAME_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(HIVE_TABLE_NAME_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP) != null); + ValidationUtils.checkArgument(lockConfiguration.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP) != null); + } + + private void setHiveLockConfs(HiveConf hiveConf) { + hiveConf.set("hive.support.concurrency", "true"); + hiveConf.set("hive.lock.manager", "org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager"); + hiveConf.set("hive.lock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP)); + hiveConf.set("hive.unlock.numretries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_NUM_RETRIES_PROP)); + hiveConf.set("hive.lock.sleep.between.retries", lockConfiguration.getConfig().getString(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP)); + hiveConf.set("hive.zookeeper.quorum", lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)); + hiveConf.set("hive.zookeeper.client.port", lockConfiguration.getConfig().getString(ZK_PORT_PROP)); + hiveConf.set("hive.zookeeper.session.timeout", lockConfiguration.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP)); + } + + private String generateLogSuffixString() { + return StringUtils.join(" database ", databaseName, " and ", "table ", tableName); + } + + protected String generateLogStatement(org.apache.hudi.common.lock.LockState state, String suffix) { + return StringUtils.join(state.name(), " lock at", suffix); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java new file mode 100644 index 0000000000000..51f7814c3d1e5 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveMetastoreLockProvider.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hive; + +import org.apache.hadoop.hive.metastore.api.DataOperationType; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.hive.testutils.HiveTestUtil; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS; +import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * For all tests, we need to set LockComponent.setOperationType(DataOperationType.NO_TXN). + * This is needed because of this -> https://github.com/apache/hive/blob/master/standalone-metastore + * /metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java#L2892 + * Unless this is set, we cannot use HiveMetastore server in tests for locking use-cases. + */ +public class TestHiveMetastoreLockProvider { + + private static Connection connection; + private static LockComponent lockComponent = new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, "testdb"); + private static LockConfiguration lockConfiguration; + + @BeforeAll + public static void init() throws Exception { + HiveTestUtil.setUp(); + createHiveConnection(); + connection.createStatement().execute("create database if not exists testdb"); + TypedProperties properties = new TypedProperties(); + properties.setProperty(HIVE_DATABASE_NAME_PROP, "testdb"); + properties.setProperty(HIVE_TABLE_NAME_PROP, "testtable"); + properties.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES); + properties.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS); + properties.setProperty(ZK_CONNECT_URL_PROP, HiveTestUtil.getZkService().connectString()); + properties.setProperty(ZK_PORT_PROP, HiveTestUtil.getHiveConf().get("hive.zookeeper.client.port")); + properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, HiveTestUtil.getHiveConf().get("hive.zookeeper.session.timeout")); + properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS)); + properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(1000)); + lockConfiguration = new LockConfiguration(properties); + lockComponent.setTablename("testtable"); + } + + @AfterAll + public static void cleanUpClass() { + HiveTestUtil.shutdown(); + } + + @Test + public void testAcquireLock() throws Exception { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + try { + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + Assertions.fail(); + } catch (Exception e) { + // Expected since lock is already acquired + } + lockProvider.unlock(); + // try to lock again after unlocking + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + lockProvider.close(); + } + + @Test + public void testUnlock() throws Exception { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + lockProvider.unlock(); + // try to lock again after unlocking + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + lockProvider.close(); + } + + @Test + public void testReentrantLock() throws Exception { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + Assertions.assertTrue(lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent)); + try { + lockProvider.acquireLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS, lockComponent); + Assertions.fail(); + } catch (IllegalArgumentException e) { + // expected + } + lockProvider.unlock(); + } + + @Test + public void testUnlockWithoutLock() { + HiveMetastoreLockProvider lockProvider = new HiveMetastoreLockProvider(lockConfiguration, HiveTestUtil.getHiveConf()); + lockComponent.setOperationType(DataOperationType.NO_TXN); + lockProvider.unlock(); + } + + private static void createHiveConnection() { + if (connection == null) { + try { + Class.forName("org.apache.hive.jdbc.HiveDriver"); + } catch (ClassNotFoundException e) { + throw new RuntimeException(); + } + try { + connection = DriverManager.getConnection("jdbc:hive2://127.0.0.1:9999/"); + } catch (SQLException e) { + throw new HoodieHiveSyncException("Cannot create hive connection ", e); + } + } + } + +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java index 2090b65de9062..ca7aa7a02ce49 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java @@ -148,7 +148,11 @@ public void stop() { hadoopConf = null; } - private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { + public HiveServer2 getHiveServer() { + return hiveServer; + } + + public HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { conf.set("hive.metastore.local", "false"); conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort); conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 5feca25b134be..80a681db40e07 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -86,6 +86,7 @@ public class HiveTestUtil { private static ZooKeeperServer zkServer; private static HiveServer2 hiveServer; private static HiveTestService hiveTestService; + private static ZookeeperTestService zkService; private static Configuration configuration; public static HiveSyncConfig hiveSyncConfig; private static DateTimeFormatter dtfOut; @@ -99,7 +100,7 @@ public static void setUp() throws IOException, InterruptedException { configuration = service.getHadoopConf(); } if (zkServer == null) { - ZookeeperTestService zkService = new ZookeeperTestService(configuration); + zkService = new ZookeeperTestService(configuration); zkServer = zkService.start(); } if (hiveServer == null) { @@ -145,6 +146,18 @@ public static HiveConf getHiveConf() { return hiveServer.getHiveConf(); } + public static HiveServer2 getHiveServer() { + return hiveServer; + } + + public static ZooKeeperServer getZkServer() { + return zkServer; + } + + public static ZookeeperTestService getZkService() { + return zkService; + } + public static void shutdown() { if (hiveServer != null) { hiveServer.stop(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 17eee8ee5c41d..336639c98a4a8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -650,7 +650,7 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { } // Validate what deltastreamer assumes of write-config to be really safe - ValidationUtils.checkArgument(config.isInlineCompaction() == cfg.isInlineCompactionEnabled(), + ValidationUtils.checkArgument(config.inlineCompactionEnabled() == cfg.isInlineCompactionEnabled(), String.format("%s should be set to %s", INLINE_COMPACT_PROP, cfg.isInlineCompactionEnabled())); ValidationUtils.checkArgument(!config.shouldAutoCommit(), String.format("%s should be set to %s", HOODIE_AUTO_COMMIT_PROP, autoCommit)); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 7522c2d29474b..8f3e045217cd7 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -18,6 +18,8 @@ package org.apache.hudi.utilities.functional; +import java.util.ConcurrentModificationException; +import java.util.concurrent.ExecutorService; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.TypedProperties; @@ -79,6 +81,7 @@ import org.apache.spark.streaming.kafka010.KafkaTestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -100,6 +103,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -120,6 +124,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { private static final String PROPS_FILENAME_TEST_CSV = "test-csv-dfs-source.properties"; protected static final String PROPS_FILENAME_TEST_PARQUET = "test-parquet-dfs-source.properties"; private static final String PROPS_FILENAME_TEST_JSON_KAFKA = "test-json-kafka-dfs-source.properties"; + private static final String PROPS_FILENAME_TEST_MULTI_WRITER = "test-multi-writer.properties"; private static final String FIRST_PARQUET_FILE_NAME = "1.parquet"; private static String PARQUET_SOURCE_ROOT; private static String JSON_KAFKA_SOURCE_ROOT; @@ -270,6 +275,34 @@ protected static void populateCommonHiveProps(TypedProperties props) { MultiPartKeysValueExtractor.class.getName()); } + protected static TypedProperties prepareMultiWriterProps(String propsFileName) throws IOException { + TypedProperties props = new TypedProperties(); + populateAllCommonProps(props); + + props.setProperty("include", "sql-transformer.properties"); + props.setProperty("hoodie.datasource.write.keygenerator.class", TestGenerator.class.getName()); + props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there"); + props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc"); + props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc"); + + props.setProperty("include", "base.properties"); + props.setProperty("hoodie.write.concurrency.mode", "optimistic_concurrency_control"); + props.setProperty("hoodie.failed.writes.cleaner.policy", "LAZY"); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); + props.setProperty("hoodie.writer.lock.hivemetastore.database", "testdb1"); + props.setProperty("hoodie.writer.lock.hivemetastore.table", "table1"); + props.setProperty("hoodie.writer.lock.zookeeper.url", "127.0.0.1"); + props.setProperty("hoodie.writer.lock.zookeeper.port", "2828"); + props.setProperty("hoodie.writer.lock.wait_time_ms", "1200000"); + props.setProperty("hoodie.writer.lock.num_retries", "10"); + props.setProperty("hoodie.writer.lock.lock_key", "test_table"); + props.setProperty("hoodie.writer.lock.zookeeper.zk_base_path", "/test"); + + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + propsFileName); + return props; + } + @AfterAll public static void cleanupClass() { UtilitiesTestBase.cleanupClass(); @@ -398,6 +431,22 @@ static void assertAtleastNDeltaCommits(int minExpected, String tablePath, FileSy assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } + static void assertAtleastNCompactionCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numCompactionCommits = (int) timeline.getInstants().count(); + assertTrue(minExpected <= numCompactionCommits, "Got=" + numCompactionCommits + ", exp >=" + minExpected); + } + + static void assertAtleastNDeltaCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numDeltaCommits = (int) timeline.getInstants().count(); + assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); + } + static String assertCommitMetadata(String expected, String tablePath, FileSystem fs, int totalCommits) throws IOException { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); @@ -406,14 +455,14 @@ static String assertCommitMetadata(String expected, String tablePath, FileSystem HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class); assertEquals(totalCommits, timeline.countInstants()); - assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY)); + assertEquals(expected, commitMetadata.getMetadata(CHECKPOINT_KEY)); return lastInstant.getTimestamp(); } - static void waitTillCondition(Function condition, long timeoutInSecs) throws Exception { + static void waitTillCondition(Function condition, Future dsFuture, long timeoutInSecs) throws Exception { Future res = Executors.newSingleThreadExecutor().submit(() -> { boolean ret = false; - while (!ret) { + while (!ret && !dsFuture.isDone()) { try { Thread.sleep(3000); ret = condition.apply(true); @@ -649,6 +698,21 @@ public void testUpsertsMORContinuousMode() throws Exception { testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor"); } + @Test + public void testUpsertsCOWContinuousModeWithMultipleWriters() throws Exception { + testUpsertsContinuousModeWithMultipleWriters(HoodieTableType.COPY_ON_WRITE, "continuous_cow_mulitwriter"); + } + + @Test + public void testUpsertsMORContinuousModeWithMultipleWriters() throws Exception { + testUpsertsContinuousModeWithMultipleWriters(HoodieTableType.MERGE_ON_READ, "continuous_mor_mulitwriter"); + } + + @Test + public void testLatestCheckpointCarryOverWithMultipleWriters() throws Exception { + testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType.COPY_ON_WRITE, "continuous_cow_checkpoint"); + } + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception { String tableBasePath = dfsBasePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode @@ -673,6 +737,215 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir }); } + private void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableType, String tempDir) throws Exception { + // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts + String tableBasePath = dfsBasePath + "/" + tempDir; + // enable carrying forward latest checkpoint + TypedProperties props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + HoodieDeltaStreamer.Config cfgIngestionJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgIngestionJob.continuousMode = true; + cfgIngestionJob.tableType = tableType.name(); + cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer ingestionJob = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + + // Prepare base dataset with some commits + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, (r) -> { + if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { + TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, dfs); + TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, dfs); + } else { + TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, dfs); + } + TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + return true; + }); + + // create a backfill job + HoodieDeltaStreamer.Config cfgBackfillJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgBackfillJob.continuousMode = false; + cfgBackfillJob.tableType = tableType.name(); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc); + + // re-init ingestion job to start sync service + HoodieDeltaStreamer ingestionJob2 = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + + // run ingestion & backfill in parallel, create conflict and fail one + runJobsInParallel(tableBasePath, tableType, totalRecords, ingestionJob2, + cfgIngestionJob, backfillJob, cfgBackfillJob, true); + + // create new ingestion & backfill job config to generate only INSERTS to avoid conflict + props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + props.setProperty("hoodie.test.source.generate.inserts", "true"); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + cfgBackfillJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, + Arrays.asList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgBackfillJob.continuousMode = false; + cfgBackfillJob.tableType = tableType.name(); + meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + + cfgIngestionJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgIngestionJob.continuousMode = true; + cfgIngestionJob.tableType = tableType.name(); + cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + // re-init ingestion job + HoodieDeltaStreamer ingestionJob3 = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + // re-init backfill job + HoodieDeltaStreamer backfillJob2 = new HoodieDeltaStreamer(cfgBackfillJob, jsc); + + // run ingestion & backfill in parallel, avoid conflict and succeed both + runJobsInParallel(tableBasePath, tableType, totalRecords, ingestionJob3, + cfgIngestionJob, backfillJob2, cfgBackfillJob, false); + } + + private void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType, String tempDir) throws Exception { + // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts + String tableBasePath = dfsBasePath + "/" + tempDir; + // enable carrying forward latest checkpoint + TypedProperties props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + HoodieDeltaStreamer.Config cfgIngestionJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgIngestionJob.continuousMode = true; + cfgIngestionJob.tableType = tableType.name(); + cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer ingestionJob = new HoodieDeltaStreamer(cfgIngestionJob, jsc); + + // Prepare base dataset with some commits + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, (r) -> { + if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { + TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, dfs); + TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, dfs); + } else { + TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, dfs); + } + TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + return true; + }); + + // create a backfill job with checkpoint from the first instant + HoodieDeltaStreamer.Config cfgBackfillJob = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, + Arrays.asList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_MULTI_WRITER, false); + cfgBackfillJob.continuousMode = false; + cfgBackfillJob.tableType = tableType.name(); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieCommitMetadata commitMetadataForFirstInstant = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + + // get current checkpoint after preparing base dataset with some commits + HoodieCommitMetadata commitMetadataForLastInstant = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + String lastCheckpointBeforeParallelBackfill = commitMetadataForLastInstant.getMetadata(CHECKPOINT_KEY); + + // run the backfill job, enable overriding checkpoint from the latest commit + props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); + props.setProperty("hoodie.writer.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.writer.lock.filesystem.path", tableBasePath); + props.setProperty("hoodie.write.meta.key.prefixes", CHECKPOINT_KEY); + UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); + + // reset checkpoint to first instant to simulate a random checkpoint for backfill job + // checkpoint will move from 00000 to 00001 for this backfill job + cfgBackfillJob.checkpoint = commitMetadataForFirstInstant.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc); + backfillJob.sync(); + + // check if the checkpoint is carried over + timeline = meta.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + commitMetadataForLastInstant = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + String lastCheckpointAfterParallelBackfill = commitMetadataForLastInstant.getMetadata(CHECKPOINT_KEY); + Assertions.assertEquals(lastCheckpointBeforeParallelBackfill, lastCheckpointAfterParallelBackfill); + } + + private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, int totalRecords, + HoodieDeltaStreamer ingestionJob, HoodieDeltaStreamer.Config cfgIngestionJob, HoodieDeltaStreamer backfillJob, + HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict) throws Exception { + ExecutorService service = Executors.newFixedThreadPool(2); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + String lastSuccessfulCommit = timeline.lastInstant().get().getTimestamp(); + // Condition for parallel ingestion job + Function conditionForRegularIngestion = (r) -> { + if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { + TestHelpers.assertAtleastNDeltaCommitsAfterCommit(3, lastSuccessfulCommit, tableBasePath, dfs); + } else { + TestHelpers.assertAtleastNCompactionCommitsAfterCommit(3, lastSuccessfulCommit, tableBasePath, dfs); + } + TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); + return true; + }; + + try { + Future regularIngestionJobFuture = service.submit(() -> { + try { + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, conditionForRegularIngestion); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + }); + Future backfillJobFuture = service.submit(() -> { + try { + backfillJob.sync(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + }); + backfillJobFuture.get(); + regularIngestionJobFuture.get(); + if (expectConflict) { + Assertions.fail("Failed to handle concurrent writes"); + } + } catch (Exception e) { + /** + * Need to perform getMessage().contains since the exception coming + * from {@link org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.DeltaSyncService} gets wrapped many times into RuntimeExceptions. + */ + if (expectConflict && e.getCause().getMessage().contains(ConcurrentModificationException.class.getName())) { + // expected ConcurrentModificationException since ingestion & backfill will have overlapping writes + } else { + throw e; + } + } + } + private void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { try { @@ -682,7 +955,7 @@ private void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer } }); - TestHelpers.waitTillCondition(condition, 240); + TestHelpers.waitTillCondition(condition, dsFuture, 240); ds.shutdownGracefully(); dsFuture.get(); } @@ -1369,4 +1642,13 @@ public Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset rowDataset, + TypedProperties properties) { + return rowDataset; + } + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 946db125289fe..6efd4680c4117 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; @@ -92,6 +93,7 @@ public class UtilitiesTestBase { protected transient SQLContext sqlContext; protected static HiveServer2 hiveServer; protected static HiveTestService hiveTestService; + protected static ZookeeperTestService zookeeperTestService; private static ObjectMapper mapper = new ObjectMapper(); @BeforeAll @@ -105,6 +107,7 @@ public static void initClass() throws Exception { public static void initClass(boolean startHiveService) throws Exception { hdfsTestService = new HdfsTestService(); + zookeeperTestService = new ZookeeperTestService(hdfsTestService.getHadoopConf()); dfsCluster = hdfsTestService.start(true); dfs = dfsCluster.getFileSystem(); dfsBasePath = dfs.getWorkingDirectory().toString(); @@ -114,6 +117,7 @@ public static void initClass(boolean startHiveService) throws Exception { hiveServer = hiveTestService.start(); clearHiveDb(); } + zookeeperTestService.start(); } @AfterAll @@ -127,6 +131,9 @@ public static void cleanupClass() { if (hiveTestService != null) { hiveTestService.stop(); } + if (zookeeperTestService != null) { + zookeeperTestService.stop(); + } } @BeforeEach diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index 84416933f7010..524591dd7bca5 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -126,6 +126,9 @@ protected static Stream fetchNextBatch(TypedProperties props, int } Stream insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .map(AbstractBaseTestSource::toGenericRecord); + if (Boolean.valueOf(props.getOrDefault("hoodie.test.source.generate.inserts", "false").toString())) { + return insertStream; + } return Stream.concat(deleteStream, Stream.concat(updateStream, insertStream)); } diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index db2635af2603a..c24021568bdc7 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -153,6 +153,9 @@ com.fasterxml.jackson.dataformat:jackson-dataformat-yaml org.apache.htrace:htrace-core + org.apache.curator:curator-framework + org.apache.curator:curator-client + org.apache.curator:curator-recipes @@ -641,5 +644,24 @@ 0.9.3 + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index ba91f54987500..e20123e5c664b 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -112,7 +112,10 @@ org.apache.hbase:hbase-protocol org.apache.hbase:hbase-server org.apache.htrace:htrace-core - commons-codec:commons-codec + org.apache.curator:curator-framework + org.apache.curator:curator-client + org.apache.curator:curator-recipes + commons-codec:commons-codec @@ -352,6 +355,25 @@ ${hbase.version} + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 1a352f3647cce..6257e97586d65 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -123,6 +123,9 @@ org.apache.hbase:hbase-protocol org.apache.hbase:hbase-server org.apache.htrace:htrace-core + org.apache.curator:curator-framework + org.apache.curator:curator-client + org.apache.curator:curator-recipes commons-codec:commons-codec @@ -347,6 +350,25 @@ hbase-protocol ${hbase.version} + + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + diff --git a/pom.xml b/pom.xml index 569ce7b50e55a..ad726c0d76b42 100644 --- a/pom.xml +++ b/pom.xml @@ -143,6 +143,7 @@ compile org.apache.hudi. true + 2.7.1 @@ -907,6 +908,25 @@ test + + + org.apache.curator + curator-framework + ${zk-curator.version} + + + + org.apache.curator + curator-client + ${zk-curator.version} + + + + org.apache.curator + curator-recipes + ${zk-curator.version} + + org.junit.jupiter junit-jupiter-api