From 79feeb3c49a0f07c4c707fb9a4513423d8fafcd6 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 15:42:59 +0800 Subject: [PATCH 01/51] need more test --- .../transaction/TransactionManager.java | 21 ++ .../client/transaction/lock/LockManager.java | 28 +++ .../apache/hudi/config/HoodieLockConfig.java | 49 ++++- .../apache/hudi/config/HoodieWriteConfig.java | 25 +++ .../org/apache/hudi/io/HoodieWriteHandle.java | 57 +++++- .../hudi/table/marker/DirectWriteMarkers.java | 24 +++ ...erBasedEarlyConflictDetectionStrategy.java | 56 +++++ .../TimelineServerBasedWriteMarkers.java | 81 +++++++- .../hudi/table/marker/WriteMarkers.java | 16 ++ .../client/TestHoodieClientMultiWriter.java | 149 +++++++++++++- ...erBasedEarlyConflictDetectionStrategy.java | 109 ++++++++++ .../HoodieEarlyConflictDetectionStrategy.java | 4 + ...erBasedEarlyConflictDetectionStrategy.java | 34 +++ ...erBasedEarlyConflictDetectionStrategy.java | 23 +++ .../common/table/marker/MarkerOperation.java | 6 + .../apache/hudi/common/util/MarkerUtils.java | 56 ++++- ...HoodieEarlyConflictDetectionException.java | 43 ++++ .../hudi/timeline/service/RequestHandler.java | 8 +- .../service/handlers/MarkerHandler.java | 53 ++++- ...eMarkerEarlyConflictDetectionStrategy.java | 61 ++++++ .../marker/MarkerCheckerRunnable.java | 193 ++++++++++++++++++ 21 files changed, 1074 insertions(+), 22 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/exception/HoodieEarlyConflictDetectionException.java create mode 100644 hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java create mode 100644 hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java 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 index aef1fee5e079..d43d4cf419ec 100644 --- 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 @@ -45,6 +45,11 @@ public TransactionManager(HoodieWriteConfig config, FileSystem fs) { this.isOptimisticConcurrencyControlEnabled = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); } + public TransactionManager(HoodieWriteConfig config, FileSystem fs, String partitionPath, String fileId) { + this.lockManager = new LockManager(config, fs, partitionPath, fileId); + this.isOptimisticConcurrencyControlEnabled = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + } + public void beginTransaction(Option newTxnOwnerInstant, Option lastCompletedTxnOwnerInstant) { if (isOptimisticConcurrencyControlEnabled) { @@ -57,6 +62,14 @@ public void beginTransaction(Option newTxnOwnerInstant, } } + public void beginTransaction(String partitionPath, String fileId) { + if (isOptimisticConcurrencyControlEnabled) { + LOG.info("Transaction starting for " + partitionPath + "/" + fileId); + lockManager.lock(); + LOG.info("Transaction started for " + partitionPath + "/" + fileId); + } + } + public void endTransaction(Option currentTxnOwnerInstant) { if (isOptimisticConcurrencyControlEnabled) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant); @@ -67,6 +80,14 @@ public void endTransaction(Option currentTxnOwnerInstant) { } } + public void endTransaction(String filePath) { + if (isOptimisticConcurrencyControlEnabled) { + LOG.info("Transaction ending with transaction for " + filePath); + lockManager.unlock(); + LOG.info("Transaction ended with transaction for " + filePath); + } + } + private synchronized boolean reset(Option callerInstant, Option newTxnOwnerInstant, Option lastCompletedTxnOwnerInstant) { 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 index ca15c4fdc2a1..4a20d08bd49a 100644 --- 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 @@ -23,6 +23,7 @@ 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.config.TypedProperties; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieLockConfig; @@ -57,6 +58,33 @@ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { Long.parseLong(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS.defaultValue())); } + /** + * Try to have a lock at partitionPath + fileID level for different write handler. + * @param writeConfig + * @param fs + * @param partitionPath + * @param fileId + */ + public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, String partitionPath, String fileId) { + this.writeConfig = writeConfig; + this.hadoopConf = new SerializableConfiguration(fs.getConf()); + TypedProperties props = refreshLockConfig(writeConfig, partitionPath + "/" + fileId); + this.lockConfiguration = new LockConfiguration(props); + maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, + Integer.parseInt(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_NUM_RETRIES.defaultValue())); + maxWaitTimeInMs = lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, + Long.parseLong(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS.defaultValue())); + } + + /** + * rebuild lock related configs, only support ZK related lock for now. + */ + private TypedProperties refreshLockConfig(HoodieWriteConfig writeConfig, String key) { + TypedProperties props = new TypedProperties(writeConfig.getProps()); + props.setProperty(LockConfiguration.ZK_LOCK_KEY_PROP_KEY, key); + return props; + } + public void lock() { if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { LockProvider lockProvider = getLockProvider(); 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 index 9ea28fbbd42e..75dd505d9dc6 100644 --- 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 @@ -180,7 +180,34 @@ public class HoodieLockConfig extends HoodieConfig { .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .sinceVersion("0.8.0") .withDocumentation("Lock provider class name, this should be subclass of " - + "org.apache.hudi.client.transaction.ConflictResolutionStrategy"); + + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy"); + + // Pluggable strategies to use when early conflict detection + public static final ConfigProperty EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME = ConfigProperty + .key(LOCK_PREFIX + "early.conflict.detection.strategy") + .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) + .sinceVersion("0.12.0") + .withDocumentation("early conflict detection class name, this should be subclass of " + + "oorg.apache.hudi.common.model.HoodieEarlyConflictDetectionStrategy"); + + public static final ConfigProperty EARLY_CONFLICT_DETECTION_ENABLE = ConfigProperty + .key(LOCK_PREFIX + "early.conflict.detection.enable") + .defaultValue(false) + .sinceVersion("0.12.0") + .withDocumentation("Enable early conflict detection based on markers. It will try to detect writing conflict before create markers and fast fail" + + " which will release cluster resources as soon as possible."); + + public static final ConfigProperty MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = ConfigProperty + .key(LOCK_PREFIX + "early.conflict.async.checker.batch.interval") + .defaultValue(30000L) + .sinceVersion("0.12.0") + .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The time to delay first async marker conflict checking."); + + public static final ConfigProperty MARKER_CONFLICT_CHECKER_PERIOD = ConfigProperty + .key(LOCK_PREFIX + "early.conflict.async.checker.period") + .defaultValue(30000L) + .sinceVersion("0.12.0") + .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking."); /** @deprecated Use {@link #WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME} and its methods instead */ @Deprecated @@ -304,6 +331,26 @@ public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutio return this; } + public HoodieLockConfig.Builder withEarlyConflictDetectionEnable(boolean enable) { + lockConfig.setValue(EARLY_CONFLICT_DETECTION_ENABLE, String.valueOf(enable)); + return this; + } + + public HoodieLockConfig.Builder withMarkerConflictCheckerBatchInterval(long interval) { + lockConfig.setValue(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, String.valueOf(interval)); + return this; + } + + public HoodieLockConfig.Builder withMarkerConflictCheckerPeriod(long period) { + lockConfig.setValue(MARKER_CONFLICT_CHECKER_PERIOD, String.valueOf(period)); + return this; + } + + public HoodieLockConfig.Builder withEarlyConflictDetectionStrategy(String className) { + lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME, className); + return this; + } + public HoodieLockConfig build() { lockConfig.setDefaults(HoodieLockConfig.class.getName()); return lockConfig; 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 4d07097c07c8..7a8e00923199 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 @@ -30,6 +30,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieMetastoreConfig; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FileSystemRetryConfig; @@ -2027,6 +2028,14 @@ public ConflictResolutionStrategy getWriteConflictResolutionStrategy() { return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME)); } + public String getMarkerConflictCheckerBatchInterval() { + return String.valueOf(getLong(HoodieLockConfig.MARKER_CONFLICT_CHECKER_BATCH_INTERVAL)); + } + + public String getMarkerConflictCheckerPeriod() { + return String.valueOf(getLong(HoodieLockConfig.MARKER_CONFLICT_CHECKER_PERIOD)); + } + public Long getLockAcquireWaitTimeoutInMs() { return getLong(HoodieLockConfig.LOCK_ACQUIRE_WAIT_TIMEOUT_MS); } @@ -2035,6 +2044,22 @@ public WriteConcurrencyMode getWriteConcurrencyMode() { return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE)); } + public boolean isEarlyConflictDetectionEnable() { + return getBoolean(HoodieLockConfig.EARLY_CONFLICT_DETECTION_ENABLE); + } + + public boolean isEarlyConflictDetectionWithTransaction() { + return getBoolean(HoodieLockConfig.EARLY_CONFLICT_DETECTION_ENABLE); + } + + public String getEarlyConflictDetectionStrategyClassName() { + return getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME); + } + + public HoodieEarlyConflictDetectionStrategy getEarlyConflictDetectionStrategy() { + return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME)); + } + /** * Are any table services configured to run inline for both scheduling and execution? * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index b7fdbecfd56d..1a4d20fff977 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -20,11 +20,15 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.conflict.detection.HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -33,6 +37,7 @@ import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.marker.WriteMarkers; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.avro.Schema; @@ -47,6 +52,8 @@ import java.util.Collections; import java.util.List; import java.util.HashMap; +import java.util.Set; +import java.util.stream.Collectors; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; @@ -183,8 +190,54 @@ protected Path makeNewFilePath(String partitionPath, String fileName) { * @param partitionPath Partition path */ protected void createMarkerFile(String partitionPath, String dataFileName) { - WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime) - .create(partitionPath, dataFileName, getIOType()); + WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime); + // do early conflict detection before create markers. + if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && config.isEarlyConflictDetectionEnable()) { + HoodieEarlyConflictDetectionStrategy earlyConflictDetectionStrategy = config.getEarlyConflictDetectionStrategy(); + if (earlyConflictDetectionStrategy instanceof HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy) { + createMarkerWithTransaction(earlyConflictDetectionStrategy, writeMarkers, partitionPath, dataFileName); + } else { + createMarkerWithEarlyConflictDetection(earlyConflictDetectionStrategy, writeMarkers, partitionPath, dataFileName); + } + } else { + // create marker directly + writeMarkers.create(partitionPath, dataFileName, getIOType()); + } + } + + private Option createMarkerWithEarlyConflictDetection(HoodieEarlyConflictDetectionStrategy resolutionStrategy, + WriteMarkers writeMarkers, + String partitionPath, + String dataFileName) { + Set completedCommitInstants = hoodieTable.getMetaClient().getActiveTimeline() + .getCommitsTimeline() + .filterCompletedInstants() + .getInstants() + .collect(Collectors.toSet()); + + return writeMarkers.createWithEarlyConflictDetection(partitionPath, dataFileName, getIOType(), false, resolutionStrategy, completedCommitInstants, config); + + } + + private Option createMarkerWithTransaction(HoodieEarlyConflictDetectionStrategy resolutionStrategy, + WriteMarkers writeMarkers, + String partitionPath, + String dataFileName) { + TransactionManager txnManager = new TransactionManager(config, fs, partitionPath, fileId); + try { + // Need to do transaction before create marker file when using early conflict detection + txnManager.beginTransaction(partitionPath, fileId); + return createMarkerWithEarlyConflictDetection(resolutionStrategy, writeMarkers, partitionPath, dataFileName); + + } catch (Exception e) { + LOG.warn("Exception occurs during create marker file in early conflict detection mode."); + throw e; + } finally { + // End transaction after created marker file. + txnManager.endTransaction(partitionPath + "/" + fileId); + txnManager.close(); + } } public Schema getWriterSchemaWithMetaFields() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index e81338207963..46288c673130 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -18,14 +18,20 @@ package org.apache.hudi.table.marker; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; @@ -155,6 +161,24 @@ protected Option create(String partitionPath, String dataFileName, IOType return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); } + @Override + public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, + HoodieEarlyConflictDetectionStrategy resolutionStrategy, + Set completedCommitInstants, HoodieWriteConfig config) { + HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) resolutionStrategy; + HoodieTableMetaClient metaClient = + HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) + .setProperties(config.getProps()).build(); + + if (strategy.hasMarkerConflict(basePath, fs, partitionPath, dataFileName, instantTime, completedCommitInstants, metaClient)) { + strategy.resolveMarkerConflict(basePath, partitionPath, dataFileName); + } + return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); + } + private Option create(Path markerPath, boolean checkIfExists) { HoodieTimer timer = new HoodieTimer().startTimer(); Path dirPath = markerPath.getParent(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..5539d473a1e5 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.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.table.marker; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ConcurrentModificationException; +import java.util.Set; + +/** + * This strategy is used for direct marker writers, trying to do early conflict detection. + * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. + */ +public class SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.class); + + @Override + public boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, + Set completedCommitInstants, HoodieTableMetaClient metaClient) { + try { + return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(metaClient, completedCommitInstants, fileId); + } catch (IOException e) { + LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); + throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); + } + } + + @Override + public void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { + throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 4879e0bc60c9..3d5647ec88f6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -18,12 +18,16 @@ package org.apache.hudi.table.marker; +import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieRemoteException; import org.apache.hudi.table.HoodieTable; @@ -38,6 +42,7 @@ import java.io.IOException; import java.util.Collections; +import java.util.ConcurrentModificationException; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -48,6 +53,12 @@ import static org.apache.hudi.common.table.marker.MarkerOperation.CREATE_MARKER_URL; import static org.apache.hudi.common.table.marker.MarkerOperation.DELETE_MARKER_DIR_URL; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKERS_DIR_EXISTS_URL; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_BASEPATH_PARAM; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_BATCH_INTERVAL; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_ENABLE; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_PERIOD; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_DIR_PATH_PARAM; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_NAME_PARAM; @@ -132,14 +143,54 @@ protected Option create(String partitionPath, String dataFileName, IOType HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = new HashMap<>(); + Map paramsMap = initConfigMap(partitionPath, markerFileName); + boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); + LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName + + " in " + timer.endTimer() + " ms"); + if (success) { + return Option.of(new Path(FSUtils.getPartitionPath(markerDirPath, partitionPath), markerFileName)); + } else { + return Option.empty(); + } + } + + @Override + public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, + HoodieEarlyConflictDetectionStrategy resolutionStrategy, + Set completedCommitInstants, HoodieWriteConfig config) { + HoodieTimer timer = new HoodieTimer().startTimer(); + String markerFileName = getMarkerFileName(dataFileName, type); + Map paramsMap = initConfigMap(partitionPath, markerFileName); + + paramsMap.put(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, config.getMarkerConflictCheckerBatchInterval()); + paramsMap.put(MARKER_CONFLICT_CHECKER_PERIOD, config.getMarkerConflictCheckerPeriod()); paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); - if (StringUtils.isNullOrEmpty(partitionPath)) { - paramsMap.put(MARKER_NAME_PARAM, markerFileName); + paramsMap.put(MARKER_BASEPATH_PARAM, basePath); + paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs())); + paramsMap.put(MARKER_CONFLICT_CHECKER_ENABLE, String.valueOf(config.isEarlyConflictDetectionEnable())); + paramsMap.put(MARKER_CONFLICT_CHECKER_STRATEGY, config.getEarlyConflictDetectionStrategyClassName()); + + boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); + + LOG.info("[timeline-server-based] Created marker file with early conflict detection " + partitionPath + "/" + markerFileName + + " in " + timer.endTimer() + " ms"); + + if (success) { + return Option.of(new Path(FSUtils.getPartitionPath(markerDirPath, partitionPath), markerFileName)); } else { - paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName); + // this failed may due to early conflict detection, so we need to throw out. + throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); } + } + /** + * execute create marker request with specific parasMap + * @param paramsMap + * @param partitionPath + * @param markerFileName + * @return + */ + private boolean executeCreateMarkerRequest(Map paramsMap, String partitionPath, String markerFileName) { boolean success; try { success = executeRequestToTimelineServer( @@ -148,13 +199,25 @@ protected Option create(String partitionPath, String dataFileName, IOType } catch (IOException e) { throw new HoodieRemoteException("Failed to create marker file " + partitionPath + "/" + markerFileName, e); } - LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName - + " in " + timer.endTimer() + " ms"); - if (success) { - return Option.of(new Path(FSUtils.getPartitionPath(markerDirPath, partitionPath), markerFileName)); + return success; + } + + /** + * init create marker related config maps. + * @param partitionPath + * @param markerFileName + * @return + */ + private Map initConfigMap(String partitionPath, String markerFileName) { + + Map paramsMap = new HashMap<>(); + paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); + if (StringUtils.isNullOrEmpty(partitionPath)) { + paramsMap.put(MARKER_NAME_PARAM, markerFileName); } else { - return Option.empty(); + paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName); } + return paramsMap; } private T executeRequestToTimelineServer(String requestPath, Map queryParameters, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 07428dd93646..981f47052e72 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -18,13 +18,16 @@ package org.apache.hudi.table.marker; +import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.Path; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -165,4 +168,17 @@ protected Path getMarkerPath(String partitionPath, String dataFileName, IOType t * @return the marker path or empty option if already exists and {@code checkIfExists} is true */ abstract Option create(String partitionPath, String dataFileName, IOType type, boolean checkIfExists); + + /** + * Creates a marker. + * + * @param partitionPath partition path in the table + * @param dataFileName data file name + * @param type write IO type + * @param checkIfExists whether to check if the marker already exists + * @return the marker path or empty option if already exists and {@code checkIfExists} is true + */ + public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, + HoodieEarlyConflictDetectionStrategy resolutionStrategy, + Set completedCommitInstants, HoodieWriteConfig config); } 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 index 268674e78d87..858ae4a181b2 100644 --- 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 @@ -31,24 +31,32 @@ 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.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.marker.SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hadoop.fs.Path; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; +import org.apache.spark.SparkException; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -65,10 +73,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -92,6 +102,68 @@ public void clean() throws IOException { cleanupResources(); } + /** + * Test multi-writers with early conflict detect enable, including + * 1. MOR + Direct marker + * 2. COW + Direct marker + * 3. MOR + Timeline server based marker + * 4. COW + Timeline server based marker + * + * ---|---------|--------------------|--------------------------------------|-------------------------> time + * init 001 + * 002 start writing + * 003 start which has conflict with 002 + * and failed soon + * 002 commit successfully + * @param tableType + * @param markerType + * @throws Exception + */ + @ParameterizedTest + @MethodSource("configParams") + public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String tableType, String markerType) throws Exception { + if (tableType.equalsIgnoreCase(HoodieTableType.MERGE_ON_READ.name())) { + setUpMORTestTable(); + } + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); + + HoodieWriteConfig writeConfig = buildWriteConfigForEarlyConflictDetect(markerType, properties); + // Create the first commit + final String nextCommitTime1 = "001"; + createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", nextCommitTime1, 2000, true); + + final SparkRDDWriteClient client2 = getHoodieWriteClient(writeConfig); + final SparkRDDWriteClient client3 = getHoodieWriteClient(writeConfig); + + final String nextCommitTime2 = "002"; + + // start to write commit 002 + final JavaRDD writeStatusList2 = startCommitForUpdate(writeConfig, client2, nextCommitTime2, 1000); + final String nextCommitTime3 = "003"; + + // start to write commit 003 + // this commit 003 will failed quickly because early conflict detection before create marker. + assertThrows(SparkException.class, () -> { + final JavaRDD writeStatusList3 = startCommitForUpdate(writeConfig, client3, nextCommitTime3, 1000); + client3.commit(nextCommitTime3, writeStatusList3); + }, "Early conflict detected but cannot resolve conflicts for overlapping writes"); + + // start to commit 002 and success + assertDoesNotThrow(() -> { + client2.commit(nextCommitTime2, writeStatusList2); + }); + + List completedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + + assertEquals(2, completedInstant.size()); + assertTrue(completedInstant.contains(nextCommitTime1)); + assertTrue(completedInstant.contains(nextCommitTime2)); + FileIOUtils.deleteDirectory(new File(basePath)); + } + @ParameterizedTest @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws Exception { @@ -114,7 +186,7 @@ public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws E .build()).withAutoCommit(false).withProperties(properties).build(); // Create the first commit - createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", "001", 200, true); + createCommitWithBulkInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", "001", 200, true); final int threadCount = 2; final ExecutorService executors = Executors.newFixedThreadPool(2); @@ -290,7 +362,7 @@ private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType t // Create the first commit with inserts HoodieWriteConfig cfg = writeConfigBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); - createCommitWithInserts(cfg, client, "000", "001", 200, true); + createCommitWithBulkInserts(cfg, client, "000", "001", 200, true); validInstants.add("001"); // Create 2 commits with upserts createCommitWithUpserts(cfg, client, "001", "000", "002", 100); @@ -365,7 +437,7 @@ private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType t final int numRecords = 100; latchCountDownAndWait(runCountDownLatch, 30000); assertDoesNotThrow(() -> { - createCommitWithInserts(cfg, client1, "003", newCommitTime, numRecords, true); + createCommitWithBulkInserts(cfg, client1, "003", newCommitTime, numRecords, true); validInstants.add("007"); }); }); @@ -426,7 +498,7 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) .build(); // Create the first commit - createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, true); + createCommitWithBulkInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, true); // Start another inflight commit String newCommitTime = "003"; int numRecords = 100; @@ -475,7 +547,7 @@ public void testHoodieClientMultiWriterAutoCommitForConflict() throws Exception HoodieWriteConfig cfg2 = writeConfigBuilder.build(); // Create the first commit - createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 5000, false); + createCommitWithBulkInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 5000, false); // Start another inflight commit String newCommitTime1 = "003"; String newCommitTime2 = "004"; @@ -559,7 +631,7 @@ public void testHoodieClientMultiWriterAutoCommitNonConflict() throws Exception HoodieWriteConfig cfg2 = writeConfigBuilder.build(); // Create the first commit - createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, false); + createCommitWithBulkInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, false); // Start another inflight commit String newCommitTime1 = "003"; String newCommitTime2 = "004"; @@ -593,7 +665,7 @@ private void createCommitWithInsertsForPartition(HoodieWriteConfig cfg, SparkRDD assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); } - private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, + private void createCommitWithBulkInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommitTime, String newCommitTime, int numRecords, boolean doCommit) throws Exception { // Finish first base commit @@ -604,6 +676,17 @@ private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient } } + private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, + String prevCommitTime, String newCommitTime, int numRecords, + boolean doCommit) throws Exception { + // Finish first base commit + JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::insert, + false, false, numRecords); + if (doCommit) { + 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 { @@ -641,4 +724,56 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, assertNoWriteErrors(statuses); return result; } + + public static Stream configParams() { + Object[][] data = + new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}, {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name()}, + {"MERGE_ON_READ", MarkerType.DIRECT.name()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name()}}; + return Stream.of(data).map(Arguments::of); + } + + private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerType, Properties properties) { + if (markerType.equalsIgnoreCase(MarkerType.DIRECT.name())) { + return getConfigBuilder() + .withHeartbeatIntervalInMs(3600 * 1000) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withStorageType(FileSystemViewStorageType.MEMORY) + .withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoClean(false).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .withAutoArchive(false).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withMarkersType(MarkerType.DIRECT.name()) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) + .withEarlyConflictDetectionEnable(true) + .withEarlyConflictDetectionStrategy(SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()) + .withMarkerConflictCheckerBatchInterval(0) + .withMarkerConflictCheckerPeriod(100) + .build()) + .withAutoCommit(false).withProperties(properties).build(); + } else { + return getConfigBuilder() + .withStorageConfig(HoodieStorageConfig.newBuilder().parquetMaxFileSize(20 * 1024).build()) + .withHeartbeatIntervalInMs(3600 * 1000) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withStorageType(FileSystemViewStorageType.MEMORY) + .withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoClean(false).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .withAutoArchive(false).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withMarkersType(MarkerType.TIMELINE_SERVER_BASED.name()) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) + .withEarlyConflictDetectionEnable(true) + .withEarlyConflictDetectionStrategy(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()) + .withMarkerConflictCheckerBatchInterval(0) + .withMarkerConflictCheckerPeriod(100) + .build()) + .withAutoCommit(false).withProperties(properties).build(); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..0554937deb0f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -0,0 +1,109 @@ +/* + * 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.conflict.detection; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.class); + + public abstract boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String dataFileName, String instantTime, + Set completedCommitInstants, HoodieTableMetaClient metaClient); + + public abstract void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName); + + /** + * We need to do list operation here. + * In order to reduce the list pressure as much as possible, first we build path prefix in advance: '$base_path/.temp/instant_time/partition_path', + * and only list these specific partition_paths we need instead of list all the '$base_path/.temp/' + * @param basePath + * @param partitionPath + * @param fileId + * @return true if current fileID is already existed under .temp/instant_time/partition_path/.. + * @throws IOException + */ + public boolean checkMarkerConflict(String basePath, String partitionPath, String fileId, + FileSystem fs, String instantTime) throws IOException { + String tempFolderPath = getTempFolderPath(basePath); + long res = Arrays.stream(fs.listStatus(new Path(tempFolderPath))) + .parallel() + .map(FileStatus::getPath) + .filter(markerPath -> { + return !markerPath.getName().equalsIgnoreCase(instantTime); + }) + .flatMap(currentMarkerDirPath -> { + try { + Path markerPartitionPath; + if (StringUtils.isNullOrEmpty(partitionPath)) { + markerPartitionPath = currentMarkerDirPath; + } else { + markerPartitionPath = new Path(currentMarkerDirPath, partitionPath); + } + + if (!StringUtils.isNullOrEmpty(partitionPath) && !fs.exists(markerPartitionPath)) { + return Stream.empty(); + } else { + return Arrays.stream(fs.listStatus(markerPartitionPath)).parallel() + .filter((path) -> path.toString().contains(fileId)); + } + } catch (IOException e) { + throw new HoodieIOException("IOException occurs during checking marker file conflict"); + } + }).count(); + + if (res != 0L) { + LOG.warn("Detected conflict marker files: " + partitionPath + "/" + fileId + " for " + instantTime); + return true; + } + return false; + } + + private String getTempFolderPath(String basePath) { + return basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; + } + + public boolean checkCommitConflict(HoodieTableMetaClient metaClient, Set completedCommitInstants, String fileId) { + Set currentInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + currentInstants.removeAll(completedCommitInstants); + Set missingFileIDs = currentInstants.stream().flatMap(instant -> { + try { + return HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class) + .getFileIdAndRelativePaths().keySet().stream(); + } catch (Exception e) { + return Stream.empty(); + } + }).collect(Collectors.toSet()); + return missingFileIDs.contains(fileId); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..27bbbd7fd05c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java @@ -0,0 +1,4 @@ +package org.apache.hudi.common.conflict.detection; + +public interface HoodieEarlyConflictDetectionStrategy { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..08a119817f88 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java @@ -0,0 +1,34 @@ +/* + * 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.conflict.detection; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import java.util.Set; + +public abstract class HoodieTimelineServerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { + + public abstract boolean hasMarkerConflict(); + + public abstract void resolveMarkerConflict(String basePath, String markerDir, String markerName); + + public void fresh(String batchInterval, String period, String markerDir, String basePath, + String maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) {} +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..39db22687714 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -0,0 +1,23 @@ +/* + * 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.conflict.detection; + +public abstract class HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy + extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java index 94da60c39c1a..ba5625480f64 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java @@ -29,6 +29,12 @@ public class MarkerOperation implements Serializable { public static final String MARKER_DIR_PATH_PARAM = "markerdirpath"; public static final String MARKER_NAME_PARAM = "markername"; + public static final String MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = "batchinterval"; + public static final String MARKER_CONFLICT_CHECKER_PERIOD = "period"; + public static final String MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL = "heartbeatinterval"; + public static final String MARKER_BASEPATH_PARAM = "basepath"; + public static final String MARKER_CONFLICT_CHECKER_ENABLE = "HoodieEarlyConflictDetectionStrategy"; + public static final String MARKER_CONFLICT_CHECKER_STRATEGY = "earlyconflictdetectionstrategy"; // GET requests public static final String ALL_MARKERS_URL = String.format("%s/%s", BASE_URL, "all"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 555a036b9f83..2dc61d5b14a7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -39,11 +39,15 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; import java.util.function.Predicate; +import java.util.stream.Collectors; import static org.apache.hudi.common.util.FileIOUtils.closeQuietly; @@ -199,6 +203,10 @@ public static Map> readTimelineServerBasedMarkersFromFileSys * @return markers in a {@code Set} of String. */ public static Set readMarkersFromFile(Path markersFilePath, SerializableConfiguration conf) { + return readMarkersFromFile(markersFilePath, conf, false); + } + + public static Set readMarkersFromFile(Path markersFilePath, SerializableConfiguration conf, boolean ignoreException) { FSDataInputStream fsDataInputStream = null; Set markers = new HashSet<>(); try { @@ -207,10 +215,56 @@ public static Set readMarkersFromFile(Path markersFilePath, Serializable fsDataInputStream = fs.open(markersFilePath); markers = new HashSet<>(FileIOUtils.readAsUTFStringLines(fsDataInputStream)); } catch (IOException e) { - throw new HoodieIOException("Failed to read MARKERS file " + markersFilePath, e); + if (ignoreException) { + LOG.warn("IOException occurs during read MARKERS file, ", e); + } else { + throw new HoodieIOException("Failed to read MARKERS file " + markersFilePath, e); + } } finally { closeQuietly(fsDataInputStream); } return markers; } + + /** + * Reads files containing the markers written by timeline-server-based marker mechanism locally instead of using cluster Context. + * + * @param markerDir marker directory. + * @param fileSystem file system to use. + * @return A {@code Map} of file name to the set of markers stored in the file. + */ + public static Set readTimelineServerBasedMarkersFromFileSystemLocally(String markerDir, FileSystem fileSystem) { + Path dirPath = new Path(markerDir); + try { + if (fileSystem.exists(dirPath)) { + Predicate prefixFilter = fileStatus -> + fileStatus.getPath().getName().startsWith(MARKERS_FILENAME_PREFIX); + Predicate markerTypeFilter = fileStatus -> + !fileStatus.getPath().getName().equals(MARKER_TYPE_FILENAME); + + CopyOnWriteArraySet result = new CopyOnWriteArraySet<>(); + FileStatus[] fileStatuses = fileSystem.listStatus(dirPath); + List subPaths = Arrays.stream(fileStatuses) + .filter(prefixFilter.and(markerTypeFilter)) + .map(fileStatus -> fileStatus.getPath().toString()) + .collect(Collectors.toList()); + + if (subPaths.size() > 0) { + SerializableConfiguration conf = new SerializableConfiguration(fileSystem.getConf()); + subPaths.stream().parallel().forEach(subPath -> { + result.addAll(readMarkersFromFile(new Path(subPath), conf, true)); + }); + } + return result; + } + return new HashSet<>(); + } catch (Exception ioe) { + LOG.warn("IOException occurs during read TimelineServer based markers from fileSystem", ioe); + return new HashSet<>(); + } + } + + public static List getAllMarkerDir(Path tempPath, FileSystem fs) throws IOException { + return Arrays.stream(fs.listStatus(tempPath)).map(FileStatus::getPath).collect(Collectors.toList()); + } } \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieEarlyConflictDetectionException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieEarlyConflictDetectionException.java new file mode 100644 index 000000000000..c88abcb4e0d5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieEarlyConflictDetectionException.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 early conflict detected. This is a runtime (unchecked) + * exception. + *

+ */ +public class HoodieEarlyConflictDetectionException extends HoodieException { + + public HoodieEarlyConflictDetectionException(String msg) { + super(msg); + } + + public HoodieEarlyConflictDetectionException(Throwable e) { + super(e); + } + + public HoodieEarlyConflictDetectionException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index 08dadae74d25..ea99a6404709 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -468,7 +468,13 @@ private void registerMarkerAPI() { ctx.result(markerHandler.createMarker( ctx, ctx.queryParam(MarkerOperation.MARKER_DIR_PATH_PARAM, ""), - ctx.queryParam(MarkerOperation.MARKER_NAME_PARAM, ""))); + ctx.queryParam(MarkerOperation.MARKER_NAME_PARAM, ""), + ctx.queryParam(MarkerOperation.MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, ""), + ctx.queryParam(MarkerOperation.MARKER_CONFLICT_CHECKER_PERIOD, ""), + ctx.queryParam(MarkerOperation.MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, ""), + ctx.queryParam(MarkerOperation.MARKER_BASEPATH_PARAM, ""), + ctx.queryParam(MarkerOperation.MARKER_CONFLICT_CHECKER_ENABLE, ""), + ctx.queryParam(MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY, ""))); }, false)); app.post(MarkerOperation.DELETE_MARKER_DIR_URL, new ViewHandler(ctx -> { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index e793c20432f9..ca8f3c516ab1 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -18,10 +18,18 @@ package org.apache.hudi.timeline.service.handlers; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_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; + +import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.timeline.service.TimelineService; import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationDispatchingRunnable; import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationFuture; @@ -38,6 +46,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -78,9 +87,13 @@ public class MarkerHandler extends Handler { // A thread to dispatch marker creation requests to batch processing threads private final MarkerCreationDispatchingRunnable markerCreationDispatchingRunnable; private final Object firstCreationRequestSeenLock = new Object(); + private final Object earlyConflictDetectionLock = new Object(); private transient HoodieEngineContext hoodieEngineContext; private ScheduledFuture dispatchingThreadFuture; private boolean firstCreationRequestSeen; + private final ConcurrentHashMap checkers; + private String currentMarkerDir = null; + private HoodieTimelineServerBasedEarlyConflictDetectionStrategy earlyConflictDetectionStrategy; public MarkerHandler(Configuration conf, TimelineService.Config timelineServiceConfig, HoodieEngineContext hoodieEngineContext, FileSystem fileSystem, @@ -97,6 +110,7 @@ public MarkerHandler(Configuration conf, TimelineService.Config timelineServiceC this.markerCreationDispatchingRunnable = new MarkerCreationDispatchingRunnable(markerDirStateMap, batchingExecutorService); this.firstCreationRequestSeen = false; + this.checkers = new ConcurrentHashMap<>(); } /** @@ -108,6 +122,7 @@ public void stop() { } dispatchingExecutorService.shutdown(); batchingExecutorService.shutdown(); + checkers.values().forEach(ExecutorService::shutdown); } /** @@ -149,7 +164,43 @@ public boolean doesMarkerDirExist(String markerDir) { * @param markerName marker name * @return the {@code CompletableFuture} instance for the request */ - public CompletableFuture createMarker(Context context, String markerDir, String markerName) { + public CompletableFuture createMarker(Context context, String markerDir, String markerName, + String batchInterval, String period, String maxAllowableHeartbeatIntervalInMs, + String basePath, String earlyConflictDetectionEnable, + String earlyConflictDetectionClassName) { + // Step1 do early conflict detection if enable + if (Boolean.parseBoolean(earlyConflictDetectionEnable)) { + try { + synchronized (earlyConflictDetectionLock) { + if (earlyConflictDetectionStrategy == null) { + earlyConflictDetectionStrategy = ReflectionUtils.loadClass(earlyConflictDetectionClassName); + } + + if (!markerDir.equalsIgnoreCase(currentMarkerDir)) { + this.currentMarkerDir = markerDir; + Set actions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION); + Set oldInstants = viewManager.getFileSystemView(basePath) + .getTimeline() + .filterCompletedInstants() + .filter(instant -> actions.contains(instant.getAction())) + .getInstants() + .collect(Collectors.toSet()); + + earlyConflictDetectionStrategy.fresh(batchInterval, period, markerDir, basePath, maxAllowableHeartbeatIntervalInMs, fileSystem, + this, oldInstants); + } + } + + if (earlyConflictDetectionStrategy.hasMarkerConflict()) { + earlyConflictDetectionStrategy.resolveMarkerConflict(basePath, markerDir, markerName); + } + } catch (Exception ex) { + LOG.warn("Failed to create marker with early conflict detection enable", ex); + return new MarkerCreationFuture(context, markerDir, markerName); + } + } + + // Step 2 create marker LOG.info("Request: create marker " + markerDir + " " + markerName); MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName); // Add the future to the list diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..b0b1d412f55b --- /dev/null +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -0,0 +1,61 @@ +/* + * 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.timeline.service.handlers.marker; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; +import org.apache.hudi.timeline.service.handlers.MarkerHandler; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ConcurrentModificationException; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +public class AsyncTimelineMarkerEarlyConflictDetectionStrategy extends HoodieTimelineServerBasedEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class); + private AtomicBoolean hasConflict = new AtomicBoolean(false); + private ScheduledExecutorService markerChecker; + + @Override + public boolean hasMarkerConflict() { + return hasConflict.get(); + } + + @Override + public void resolveMarkerConflict(String basePath, String markerDir, String markerName) { + throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); + } + + public void fresh(String batchInterval, String period, String markerDir, String basePath, + String maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) { + if (markerChecker != null) { + markerChecker.shutdown(); + } + hasConflict.compareAndSet(true, false); + markerChecker = Executors.newSingleThreadScheduledExecutor(); + markerChecker.scheduleAtFixedRate(new MarkerCheckerRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, + fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants), Long.parseLong(batchInterval), Long.parseLong(period), TimeUnit.MILLISECONDS); + } +} diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java new file mode 100644 index 000000000000..05479e73e888 --- /dev/null +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java @@ -0,0 +1,193 @@ +/* + * 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.timeline.service.handlers.marker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.MarkerUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.timeline.service.handlers.MarkerHandler; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class MarkerCheckerRunnable implements Runnable { + private static final Logger LOG = LogManager.getLogger(MarkerCheckerRunnable.class); + + private MarkerHandler markerHandler; + private String markerDir; + private String basePath; + private FileSystem fs; + private AtomicBoolean hasConflict; + private long maxAllowableHeartbeatIntervalInMs; + private Set oldInstants; + + public MarkerCheckerRunnable(AtomicBoolean hasConflict, MarkerHandler markerHandler, String markerDir, + String basePath, FileSystem fileSystem, long maxAllowableHeartbeatIntervalInMs, + Set oldInstants) { + this.markerHandler = markerHandler; + this.markerDir = markerDir; + this.basePath = basePath; + this.fs = fileSystem; + this.hasConflict = hasConflict; + this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; + this.oldInstants = oldInstants; + } + + @Override + public void run() { + try { + if (!fs.exists(new Path(markerDir))) { + return; + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + Set currentInstantAllMarkers = markerHandler.getAllMarkers(markerDir); + Path tempPath = new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME); + + List instants = MarkerUtils.getAllMarkerDir(tempPath, fs); + List candidate = getCandidateInstants(instants, markerDirToInstantTime(markerDir)); + Set tableMarkers = candidate.stream().flatMap(instant -> { + return MarkerUtils.readTimelineServerBasedMarkersFromFileSystemLocally(instant, fs).stream(); + }).collect(Collectors.toSet()); + + Set currentFileIDs = currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); + Set tableFilesIDs = tableMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); + + currentFileIDs.retainAll(tableFilesIDs); + + if (!currentFileIDs.isEmpty() || hasCommitConflict(currentInstantAllMarkers, basePath)) { + LOG.warn("Conflict writing detected based on markers!\n" + + "Conflict markers: " + currentInstantAllMarkers + "\n" + + "Table markers: " + tableMarkers); + hasConflict.compareAndSet(false, true); + } + LOG.info("Finish batch marker checker in " + timer.endTimer() + " ms"); + + } catch (IOException e) { + throw new HoodieIOException("IOException occurs during checking marker conflict"); + } + } + + /** + * Get Candidate Instant to do conflict checking: + * 1. Skip current writer related instant(currentInstantTime) + * 2. Skip all instants after currentInstantTime + * 3. Skip dead writers related instants based on heart-beat + * @param instants + * @return + */ + private List getCandidateInstants(List instants, String currentInstantTime) { + return instants.stream().map(Path::toString).filter(instantPath -> { + String instantTime = markerDirToInstantTime(instantPath); + return instantTime.compareToIgnoreCase(currentInstantTime) < 0; + }).filter(instantPath -> { + try { + return !isHeartbeatExpired(markerDirToInstantTime(instantPath)); + } catch (IOException e) { + return false; + } + }).collect(Collectors.toList()); + } + + /** + * Get fileID from full marker path, for example: + * 20210623/0/20210825/932a86d9-5c1d-44c7-ac99-cb88b8ef8478-0_85-15-1390_20220620181735781.parquet.marker.MERGE + * ==> get 20210623/0/20210825/932a86d9-5c1d-44c7-ac99-cb88b8ef8478-0 + * @param marker + * @return + */ + private String makerToPartitionAndFileID(String marker) { + String[] ele = marker.split("_"); + return ele[0]; + } + + /** + * Get instantTime from full marker path, for example: + * /var/folders/t3/th1dw75d0yz2x2k2qt6ys9zh0000gp/T/junit6502909693741900820/dataset/.hoodie/.temp/003 + * ==> 003 + * @param marker + * @return + */ + private static String markerDirToInstantTime(String marker) { + String[] ele = marker.split("/"); + return ele[ele.length - 1]; + } + + /** + * Use modification time as last heart beat time + * @param fs + * @param basePath + * @param instantTime + * @return + * @throws IOException + */ + public Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); + if (fs.exists(heartbeatFilePath)) { + return fs.getFileStatus(heartbeatFilePath).getModificationTime(); + } else { + // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed + return 0L; + } + } + + public boolean isHeartbeatExpired(String instantTime) throws IOException { + Long currentTime = System.currentTimeMillis(); + Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); + if (currentTime - lastHeartbeatTime > this.maxAllowableHeartbeatIntervalInMs) { + LOG.warn("Heartbeat expired, for instant: " + instantTime); + return true; + } + return false; + } + + public boolean hasCommitConflict(Set currentInstantAllMarkers, String basePath) { + Set currentFileIDs = currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); + + HoodieTableMetaClient metaClient = + HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true).build(); + + Set currentInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + + currentInstants.removeAll(oldInstants); + Set missingFileIDs = currentInstants.stream().flatMap(instant -> { + try { + return HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class) + .getFileIdAndRelativePaths().keySet().stream(); + } catch (Exception e) { + return Stream.empty(); + } + }).collect(Collectors.toSet()); + currentFileIDs.retainAll(missingFileIDs); + return !currentFileIDs.isEmpty(); + } +} From 63314398dfcd9750fb7c1594b20e0e863cc6b43b Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 17:23:33 +0800 Subject: [PATCH 02/51] tested --- .../apache/hudi/config/HoodieLockConfig.java | 2 +- .../apache/hudi/config/HoodieWriteConfig.java | 8 +- .../org/apache/hudi/io/HoodieWriteHandle.java | 2 +- .../hudi/table/marker/DirectWriteMarkers.java | 6 +- .../TimelineServerBasedWriteMarkers.java | 2 +- .../hudi/table/marker/WriteMarkers.java | 2 +- .../client/TestHoodieClientMultiWriter.java | 73 ++++++++++++++++++- ...erBasedEarlyConflictDetectionStrategy.java | 2 +- .../service/handlers/MarkerHandler.java | 13 +++- 9 files changed, 92 insertions(+), 18 deletions(-) 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 index 75dd505d9dc6..6fa612659d91 100644 --- 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 @@ -347,7 +347,7 @@ public HoodieLockConfig.Builder withMarkerConflictCheckerPeriod(long period) { } public HoodieLockConfig.Builder withEarlyConflictDetectionStrategy(String className) { - lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME, className); + lockConfig.setValue(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME, className); return this; } 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 7a8e00923199..aba6f8bd5751 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 @@ -2048,16 +2048,12 @@ public boolean isEarlyConflictDetectionEnable() { return getBoolean(HoodieLockConfig.EARLY_CONFLICT_DETECTION_ENABLE); } - public boolean isEarlyConflictDetectionWithTransaction() { - return getBoolean(HoodieLockConfig.EARLY_CONFLICT_DETECTION_ENABLE); - } - public String getEarlyConflictDetectionStrategyClassName() { - return getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME); + return getString(HoodieLockConfig.EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME); } public HoodieEarlyConflictDetectionStrategy getEarlyConflictDetectionStrategy() { - return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME)); + return ReflectionUtils.loadClass(getString(HoodieLockConfig.EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME)); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 1a4d20fff977..8497d4207505 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -216,7 +216,7 @@ private Option createMarkerWithEarlyConflictDetection(HoodieEarlyConflictD .getInstants() .collect(Collectors.toSet()); - return writeMarkers.createWithEarlyConflictDetection(partitionPath, dataFileName, getIOType(), false, resolutionStrategy, completedCommitInstants, config); + return writeMarkers.createWithEarlyConflictDetection(partitionPath, dataFileName, getIOType(), false, resolutionStrategy, completedCommitInstants, config, fileId); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index 46288c673130..23d89b943a4f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -164,7 +164,7 @@ protected Option create(String partitionPath, String dataFileName, IOType @Override public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieEarlyConflictDetectionStrategy resolutionStrategy, - Set completedCommitInstants, HoodieWriteConfig config) { + Set completedCommitInstants, HoodieWriteConfig config, String fileId) { HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) resolutionStrategy; HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(config.getBasePath()) @@ -173,8 +173,8 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setProperties(config.getProps()).build(); - if (strategy.hasMarkerConflict(basePath, fs, partitionPath, dataFileName, instantTime, completedCommitInstants, metaClient)) { - strategy.resolveMarkerConflict(basePath, partitionPath, dataFileName); + if (strategy.hasMarkerConflict(basePath, fs, partitionPath, fileId, instantTime, completedCommitInstants, metaClient)) { + strategy.resolveMarkerConflict(basePath, partitionPath, fileId); } return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 3d5647ec88f6..a26e359dd078 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -157,7 +157,7 @@ protected Option create(String partitionPath, String dataFileName, IOType @Override public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieEarlyConflictDetectionStrategy resolutionStrategy, - Set completedCommitInstants, HoodieWriteConfig config) { + Set completedCommitInstants, HoodieWriteConfig config, String fileId) { HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); Map paramsMap = initConfigMap(partitionPath, markerFileName); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 981f47052e72..c4ebcf200905 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -180,5 +180,5 @@ protected Path getMarkerPath(String partitionPath, String dataFileName, IOType t */ public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieEarlyConflictDetectionStrategy resolutionStrategy, - Set completedCommitInstants, HoodieWriteConfig config); + Set completedCommitInstants, HoodieWriteConfig config, String fileId); } 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 index 858ae4a181b2..1c1346587d04 100644 --- 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 @@ -141,10 +141,10 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta // start to write commit 002 final JavaRDD writeStatusList2 = startCommitForUpdate(writeConfig, client2, nextCommitTime2, 1000); - final String nextCommitTime3 = "003"; // start to write commit 003 // this commit 003 will failed quickly because early conflict detection before create marker. + final String nextCommitTime3 = "003"; assertThrows(SparkException.class, () -> { final JavaRDD writeStatusList3 = startCommitForUpdate(writeConfig, client3, nextCommitTime3, 1000); client3.commit(nextCommitTime3, writeStatusList3); @@ -164,6 +164,71 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta FileIOUtils.deleteDirectory(new File(basePath)); } + /** + * Test multi-writers with early conflict detect enable, including + * 1. MOR + Direct marker + * 2. COW + Direct marker + * 3. MOR + Timeline server based marker + * 4. COW + Timeline server based marker + * + * ---|---------|--------------------|--------------------------------------|-------------------------> time + * init 001 + * 002 start writing + * 003 start which has conflict with 002 + * and failed soon + * 002 commit successfully + * @param tableType + * @param markerType + * @throws Exception + */ + @ParameterizedTest + @MethodSource("configParams") + public void testHoodieClientBasicMultiWriterWithEarlyConflictDetectionCommitConflict(String tableType, String markerType) throws Exception { + if (tableType.equalsIgnoreCase(HoodieTableType.MERGE_ON_READ.name())) { + setUpMORTestTable(); + } + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); + + HoodieWriteConfig writeConfig = buildWriteConfigForEarlyConflictDetect(markerType, properties); + // Create the first commit + final String nextCommitTime1 = "001"; + createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", nextCommitTime1, 2000, true); + + final SparkRDDWriteClient client2 = getHoodieWriteClient(writeConfig); + final SparkRDDWriteClient client3 = getHoodieWriteClient(writeConfig); + + final String nextCommitTime2 = "002"; + final String nextCommitTime3 = "003"; + + // start to write commit 002 + final JavaRDD writeStatusList2 = startCommitForUpdate(writeConfig, client2, nextCommitTime2, 1000); + // start to commit 002 and success + assertDoesNotThrow(() -> { + client2.commit(nextCommitTime2, writeStatusList2); + }); + + // start to write commit 003 + final JavaRDD writeStatusList3 = startCommitForUpdate(writeConfig, client3, nextCommitTime3, 1000); + + // start to commit 003 + // this commit 003 will failed quickly because early conflict detection before create marker. + assertThrows(SparkException.class, () -> { + client3.commit(nextCommitTime3, writeStatusList3); + }, "Early conflict detected but cannot resolve conflicts for overlapping writes"); + + + + List completedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + + assertEquals(2, completedInstant.size()); + assertTrue(completedInstant.contains(nextCommitTime1)); + assertTrue(completedInstant.contains(nextCommitTime2)); + FileIOUtils.deleteDirectory(new File(basePath)); + } + @ParameterizedTest @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws Exception { @@ -726,9 +791,11 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, } public static Stream configParams() { +// Object[][] data = +// new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}, {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name()}, +// {"MERGE_ON_READ", MarkerType.DIRECT.name()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name()}}; Object[][] data = - new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}, {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name()}}; + new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}}; return Stream.of(data).map(Arguments::of); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 0554937deb0f..dbc0a7148ca8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -49,7 +49,7 @@ public abstract boolean hasMarkerConflict(String basePath, FileSystem fs, String * and only list these specific partition_paths we need instead of list all the '$base_path/.temp/' * @param basePath * @param partitionPath - * @param fileId + * @param fileId 162b13d7-9530-48cf-88a4-02241817ae0c-0_1-74-100_003.parquet * @return true if current fileID is already existed under .temp/instant_time/partition_path/.. * @throws IOException */ diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index ca8f3c516ab1..9a89460d35bb 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -21,7 +21,10 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_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; +import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; @@ -30,6 +33,7 @@ import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.timeline.service.TimelineService; import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationDispatchingRunnable; import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationFuture; @@ -196,7 +200,14 @@ public CompletableFuture createMarker(Context context, String markerDir, } } catch (Exception ex) { LOG.warn("Failed to create marker with early conflict detection enable", ex); - return new MarkerCreationFuture(context, markerDir, markerName); + MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName); + try { + future.complete(jsonifyResult( + future.getContext(), future.isSuccessful(), metricsRegistry, new ObjectMapper(), LOG)); + } catch (JsonProcessingException e) { + throw new HoodieException("Failed to JSON encode the value", e); + } + return future; } } From de69c0e93db6511c4f217305f3fc459a50e2e78f Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 17:24:19 +0800 Subject: [PATCH 03/51] tested --- .../client/TestHoodieClientMultiWriter.java | 65 ------------------- 1 file changed, 65 deletions(-) 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 index 1c1346587d04..b8b1a5463d12 100644 --- 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 @@ -164,71 +164,6 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta FileIOUtils.deleteDirectory(new File(basePath)); } - /** - * Test multi-writers with early conflict detect enable, including - * 1. MOR + Direct marker - * 2. COW + Direct marker - * 3. MOR + Timeline server based marker - * 4. COW + Timeline server based marker - * - * ---|---------|--------------------|--------------------------------------|-------------------------> time - * init 001 - * 002 start writing - * 003 start which has conflict with 002 - * and failed soon - * 002 commit successfully - * @param tableType - * @param markerType - * @throws Exception - */ - @ParameterizedTest - @MethodSource("configParams") - public void testHoodieClientBasicMultiWriterWithEarlyConflictDetectionCommitConflict(String tableType, String markerType) throws Exception { - if (tableType.equalsIgnoreCase(HoodieTableType.MERGE_ON_READ.name())) { - setUpMORTestTable(); - } - Properties properties = new Properties(); - properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); - properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); - - HoodieWriteConfig writeConfig = buildWriteConfigForEarlyConflictDetect(markerType, properties); - // Create the first commit - final String nextCommitTime1 = "001"; - createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", nextCommitTime1, 2000, true); - - final SparkRDDWriteClient client2 = getHoodieWriteClient(writeConfig); - final SparkRDDWriteClient client3 = getHoodieWriteClient(writeConfig); - - final String nextCommitTime2 = "002"; - final String nextCommitTime3 = "003"; - - // start to write commit 002 - final JavaRDD writeStatusList2 = startCommitForUpdate(writeConfig, client2, nextCommitTime2, 1000); - // start to commit 002 and success - assertDoesNotThrow(() -> { - client2.commit(nextCommitTime2, writeStatusList2); - }); - - // start to write commit 003 - final JavaRDD writeStatusList3 = startCommitForUpdate(writeConfig, client3, nextCommitTime3, 1000); - - // start to commit 003 - // this commit 003 will failed quickly because early conflict detection before create marker. - assertThrows(SparkException.class, () -> { - client3.commit(nextCommitTime3, writeStatusList3); - }, "Early conflict detected but cannot resolve conflicts for overlapping writes"); - - - - List completedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - - assertEquals(2, completedInstant.size()); - assertTrue(completedInstant.contains(nextCommitTime1)); - assertTrue(completedInstant.contains(nextCommitTime2)); - FileIOUtils.deleteDirectory(new File(basePath)); - } - @ParameterizedTest @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws Exception { From fcaaf9d6dc063aed48319e335c181e913ac9106f Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 17:24:35 +0800 Subject: [PATCH 04/51] tested --- .../org/apache/hudi/client/TestHoodieClientMultiWriter.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 index b8b1a5463d12..1b44ea50d5c1 100644 --- 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 @@ -726,11 +726,9 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, } public static Stream configParams() { -// Object[][] data = -// new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}, {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name()}, -// {"MERGE_ON_READ", MarkerType.DIRECT.name()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name()}}; Object[][] data = - new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}}; + new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}, {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name()}, + {"MERGE_ON_READ", MarkerType.DIRECT.name()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name()}}; return Stream.of(data).map(Arguments::of); } From 553fb00b590e9586ade21e86569ccb30d8dec7a9 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 18:02:21 +0800 Subject: [PATCH 05/51] fix liences --- .../HoodieEarlyConflictDetectionStrategy.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java index 27bbbd7fd05c..9c8be817aaaa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java @@ -1,3 +1,21 @@ +/* + * 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.conflict.detection; public interface HoodieEarlyConflictDetectionStrategy { From dbe3db845908d261baa5a1aa71d19e0db55816de Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 18:57:22 +0800 Subject: [PATCH 06/51] fix config --- .../java/org/apache/hudi/config/HoodieLockConfig.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) 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 index 6fa612659d91..2a56aa282591 100644 --- 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 @@ -26,6 +26,7 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.Option; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; import java.io.File; import java.io.FileReader; @@ -180,15 +181,15 @@ public class HoodieLockConfig extends HoodieConfig { .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .sinceVersion("0.8.0") .withDocumentation("Lock provider class name, this should be subclass of " - + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy"); + + "org.apache.hudi.client.transaction.ConflictResolutionStrategy"); // Pluggable strategies to use when early conflict detection public static final ConfigProperty EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME = ConfigProperty .key(LOCK_PREFIX + "early.conflict.detection.strategy") - .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) + .defaultValue(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()) .sinceVersion("0.12.0") - .withDocumentation("early conflict detection class name, this should be subclass of " - + "oorg.apache.hudi.common.model.HoodieEarlyConflictDetectionStrategy"); + .withDocumentation("Early conflict detection class name, this should be subclass of " + + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy"); public static final ConfigProperty EARLY_CONFLICT_DETECTION_ENABLE = ConfigProperty .key(LOCK_PREFIX + "early.conflict.detection.enable") From 66b7d1b75c7a201b5d108cd48a70363dd00a1795 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 18 Jul 2022 19:28:38 +0800 Subject: [PATCH 07/51] add uts --- ...rBasedEarlyConflictDetectionStrategy.java} | 4 +- ...erBasedEarlyConflictDetectionStrategy.java | 56 +++++++++++++++++++ .../client/TestHoodieClientMultiWriter.java | 4 +- 3 files changed, 60 insertions(+), 4 deletions(-) rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/{SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java => SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java} (92%) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java similarity index 92% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index 5539d473a1e5..e9291845a763 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -35,8 +35,8 @@ * This strategy is used for direct marker writers, trying to do early conflict detection. * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. */ -public class SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.class); +public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); @Override public boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java new file mode 100644 index 000000000000..8da21c6c81fb --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.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.table.marker; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.conflict.detection.HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ConcurrentModificationException; +import java.util.Set; + +/** + * This strategy is used for direct marker writers, trying to do early conflict detection. + * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. + */ +public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); + + @Override + public boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, + Set completedCommitInstants, HoodieTableMetaClient metaClient) { + try { + return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(metaClient, completedCommitInstants, fileId); + } catch (IOException e) { + LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); + throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); + } + } + + @Override + public void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { + throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); + } +} 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 index 1b44ea50d5c1..76b5dab1d9cd 100644 --- 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 @@ -42,7 +42,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.marker.SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hadoop.fs.Path; @@ -748,7 +748,7 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withMarkersType(MarkerType.DIRECT.name()) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) .withEarlyConflictDetectionEnable(true) - .withEarlyConflictDetectionStrategy(SimpleDirectMarkerDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()) + .withEarlyConflictDetectionStrategy(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()) .withMarkerConflictCheckerBatchInterval(0) .withMarkerConflictCheckerPeriod(100) .build()) From 5842dcfcb4a2d24ea3c6d982960736f9e0d5e57d Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 19 Oct 2022 14:19:50 +0800 Subject: [PATCH 08/51] merge from master && resolve conflicts --- .../org/apache/hudi/timeline/service/handlers/MarkerHandler.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 46d4c8079e8b..387842dfca1a 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -50,7 +50,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; From 645766d09d376bc46a1e0b6af770d9e84f778b24 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 19 Oct 2022 14:26:00 +0800 Subject: [PATCH 09/51] fix checkstyle --- .../src/main/java/org/apache/hudi/config/HoodieWriteConfig.java | 1 + 1 file changed, 1 insertion(+) 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 b6e847762206..34d38416f962 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 @@ -2121,6 +2121,7 @@ public String getEarlyConflictDetectionStrategyClassName() { public HoodieEarlyConflictDetectionStrategy getEarlyConflictDetectionStrategy() { return ReflectionUtils.loadClass(getString(HoodieLockConfig.EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME)); } + // misc configs public Boolean doSkipDefaultPartitionValidation() { return getBoolean(SKIP_DEFAULT_PARTITION_VALIDATION); From 5d0d05f487ed3de86f1c7382deb59bc22d5c6395 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 11 Nov 2022 11:51:46 -0800 Subject: [PATCH 10/51] Resolve conflict with master --- .../src/main/java/org/apache/hudi/io/HoodieWriteHandle.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 278bae5a9ef8..67ff2357a8f8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -22,16 +22,16 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.conflict.detection.HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.model.IOType; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -54,8 +54,8 @@ import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.HashMap; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; From c6bc22d531f6202b0db1b9cc84c91e4710510417 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 18 Nov 2022 18:41:35 +0800 Subject: [PATCH 11/51] refact abstraction --- .../transaction/TransactionManager.java | 3 +- .../client/transaction/lock/LockManager.java | 38 +++--- .../apache/hudi/config/HoodieLockConfig.java | 48 -------- .../apache/hudi/config/HoodieWriteConfig.java | 74 ++++++++++-- .../org/apache/hudi/io/HoodieIOHandle.java | 3 +- .../org/apache/hudi/io/HoodieReadHandle.java | 2 +- .../org/apache/hudi/io/HoodieWriteHandle.java | 72 +++-------- .../hudi/table/marker/DirectWriteMarkers.java | 25 ++-- ...erBasedEarlyConflictDetectionStrategy.java | 32 ++++- ...erBasedEarlyConflictDetectionStrategy.java | 114 +++++++++--------- .../TimelineServerBasedWriteMarkers.java | 7 +- .../hudi/table/marker/WriteMarkers.java | 39 +++++- .../client/TestHoodieClientMultiWriter.java | 22 ++-- ...erBasedEarlyConflictDetectionStrategy.java | 37 +++++- .../HoodieEarlyConflictDetectionStrategy.java | 2 + ...erBasedEarlyConflictDetectionStrategy.java | 14 ++- ...erBasedEarlyConflictDetectionStrategy.java | 23 ---- .../service/handlers/MarkerHandler.java | 7 +- ...eMarkerEarlyConflictDetectionStrategy.java | 13 ++ 19 files changed, 310 insertions(+), 265 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java 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 index b5c76d6d9a0f..e778f0c4adfa 100644 --- 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 @@ -80,8 +80,9 @@ public void endTransaction(Option currentTxnOwnerInstant) { } } - public void endTransaction(String filePath) { + public void endTransaction(String partitionPath, String fileId) { if (isOptimisticConcurrencyControlEnabled) { + String filePath = partitionPath + "/" + fileId; LOG.info("Transaction ending with transaction for " + filePath); lockManager.unlock(); LOG.info("Transaction ended with transaction for " + filePath); 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 index a9e1c1853bda..30cebf8647d6 100644 --- 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 @@ -18,17 +18,20 @@ package org.apache.hudi.client.transaction.lock; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.transaction.lock.metrics.HoodieLockMetrics; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieLockException; import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -44,23 +47,17 @@ 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 final int maxRetries; - private final long maxWaitTimeInMs; + private HoodieWriteConfig writeConfig; + private LockConfiguration lockConfiguration; + private SerializableConfiguration hadoopConf; + private int maxRetries; + private long maxWaitTimeInMs; private transient HoodieLockMetrics metrics; private volatile LockProvider lockProvider; public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { - this.writeConfig = writeConfig; - this.hadoopConf = new SerializableConfiguration(fs.getConf()); this.lockConfiguration = new LockConfiguration(writeConfig.getProps()); - maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, - Integer.parseInt(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_NUM_RETRIES.defaultValue())); - maxWaitTimeInMs = lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, - Long.parseLong(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS.defaultValue())); - metrics = new HoodieLockMetrics(writeConfig); + init(writeConfig, fs.getConf()); } /** @@ -71,14 +68,19 @@ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { * @param fileId */ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, String partitionPath, String fileId) { - this.writeConfig = writeConfig; - this.hadoopConf = new SerializableConfiguration(fs.getConf()); TypedProperties props = refreshLockConfig(writeConfig, partitionPath + "/" + fileId); this.lockConfiguration = new LockConfiguration(props); - maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, + init(writeConfig, fs.getConf()); + } + + private void init(HoodieWriteConfig writeConfig, Configuration conf) { + this.writeConfig = writeConfig; + this.hadoopConf = new SerializableConfiguration(conf); + this.maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, Integer.parseInt(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_NUM_RETRIES.defaultValue())); - maxWaitTimeInMs = lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, + this.maxWaitTimeInMs = lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, Long.parseLong(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS.defaultValue())); + this.metrics = new HoodieLockMetrics(writeConfig); } /** @@ -86,6 +88,10 @@ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, String partitio */ private TypedProperties refreshLockConfig(HoodieWriteConfig writeConfig, String key) { TypedProperties props = new TypedProperties(writeConfig.getProps()); + String zk_base_path = props.getProperty(LockConfiguration.ZK_BASE_PATH_PROP_KEY); + if (StringUtils.isNullOrEmpty(zk_base_path)) { + throw new HoodieNotSupportedException("Only Support ZK based lock for now."); + } props.setProperty(LockConfiguration.ZK_LOCK_KEY_PROP_KEY, key); return props; } 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 index 121890424d89..3623a04232be 100644 --- 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 @@ -26,7 +26,6 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.Option; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; import java.io.File; import java.io.FileReader; @@ -190,33 +189,6 @@ public class HoodieLockConfig extends HoodieConfig { .withDocumentation("Lock provider class name, this should be subclass of " + "org.apache.hudi.client.transaction.ConflictResolutionStrategy"); - // Pluggable strategies to use when early conflict detection - public static final ConfigProperty EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME = ConfigProperty - .key(LOCK_PREFIX + "early.conflict.detection.strategy") - .defaultValue(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()) - .sinceVersion("0.12.0") - .withDocumentation("Early conflict detection class name, this should be subclass of " - + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy"); - - public static final ConfigProperty EARLY_CONFLICT_DETECTION_ENABLE = ConfigProperty - .key(LOCK_PREFIX + "early.conflict.detection.enable") - .defaultValue(false) - .sinceVersion("0.12.0") - .withDocumentation("Enable early conflict detection based on markers. It will try to detect writing conflict before create markers and fast fail" - + " which will release cluster resources as soon as possible."); - - public static final ConfigProperty MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = ConfigProperty - .key(LOCK_PREFIX + "early.conflict.async.checker.batch.interval") - .defaultValue(30000L) - .sinceVersion("0.12.0") - .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The time to delay first async marker conflict checking."); - - public static final ConfigProperty MARKER_CONFLICT_CHECKER_PERIOD = ConfigProperty - .key(LOCK_PREFIX + "early.conflict.async.checker.period") - .defaultValue(30000L) - .sinceVersion("0.12.0") - .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking."); - /** @deprecated Use {@link #WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME} and its methods instead */ @Deprecated public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP = WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME.key(); @@ -339,26 +311,6 @@ public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutio return this; } - public HoodieLockConfig.Builder withEarlyConflictDetectionEnable(boolean enable) { - lockConfig.setValue(EARLY_CONFLICT_DETECTION_ENABLE, String.valueOf(enable)); - return this; - } - - public HoodieLockConfig.Builder withMarkerConflictCheckerBatchInterval(long interval) { - lockConfig.setValue(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, String.valueOf(interval)); - return this; - } - - public HoodieLockConfig.Builder withMarkerConflictCheckerPeriod(long period) { - lockConfig.setValue(MARKER_CONFLICT_CHECKER_PERIOD, String.valueOf(period)); - return this; - } - - public HoodieLockConfig.Builder withEarlyConflictDetectionStrategy(String className) { - lockConfig.setValue(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME, className); - return this; - } - public HoodieLockConfig.Builder withFileSystemLockPath(String path) { lockConfig.setValue(FILESYSTEM_LOCK_PATH, path); return this; 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 855aca9fd77d..6698c49377e3 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 @@ -70,9 +70,11 @@ import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; +import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.orc.CompressionKind; @@ -113,6 +115,8 @@ public class HoodieWriteConfig extends HoodieConfig { // It is here so that both the client and deltastreamer use the same reference public static final String DELTASTREAMER_CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; + public static final String CONCURRENCY_PREFIX = "hoodie.write.concurrency."; + public static final ConfigProperty TBL_NAME = ConfigProperty .key(HoodieTableConfig.HOODIE_TABLE_NAME_KEY) .noDefaultValue() @@ -513,6 +517,44 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("When table is upgraded from pre 0.12 to 0.12, we check for \"default\" partition and fail if found one. " + "Users are expected to rewrite the data in those partitions. Enabling this config will bypass this validation"); + // Pluggable strategies to use when early conflict detection + public static final ConfigProperty EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.detection.strategy") + .noDefaultValue() + .sinceVersion("0.13.0") + .withInferFunction(cfg -> { + MarkerType markerType = MarkerType.valueOf(cfg.getString(MARKERS_TYPE).toUpperCase()); + switch (markerType) { + case DIRECT: + return Option.of(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()); + case TIMELINE_SERVER_BASED: + default: + return Option.of(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()); + } + }) + .withDocumentation("Early conflict detection class name, this should be subclass of " + + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy"); + + public static final ConfigProperty EARLY_CONFLICT_DETECTION_ENABLE = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.detection.enable") + .defaultValue(false) + .sinceVersion("0.13.0") + .withDocumentation("Enable early conflict detection based on markers. It will try to detect writing conflict before create markers and fast fail" + + " which will release cluster resources as soon as possible."); + + public static final ConfigProperty MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.async.checker.batch.interval") + .defaultValue(30000L) + .sinceVersion("0.13.0") + .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The time to delay first async marker conflict checking."); + + public static final ConfigProperty MARKER_CONFLICT_CHECKER_PERIOD = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.async.checker.period") + .defaultValue(30000L) + .sinceVersion("0.13.0") + .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking."); + + private ConsistencyGuardConfig consistencyGuardConfig; private FileSystemRetryConfig fileSystemRetryConfig; @@ -2131,11 +2173,11 @@ public ConflictResolutionStrategy getWriteConflictResolutionStrategy() { } public String getMarkerConflictCheckerBatchInterval() { - return String.valueOf(getLong(HoodieLockConfig.MARKER_CONFLICT_CHECKER_BATCH_INTERVAL)); + return String.valueOf(getLong(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL)); } public String getMarkerConflictCheckerPeriod() { - return String.valueOf(getLong(HoodieLockConfig.MARKER_CONFLICT_CHECKER_PERIOD)); + return String.valueOf(getLong(MARKER_CONFLICT_CHECKER_PERIOD)); } public Long getLockAcquireWaitTimeoutInMs() { @@ -2147,15 +2189,11 @@ public WriteConcurrencyMode getWriteConcurrencyMode() { } public boolean isEarlyConflictDetectionEnable() { - return getBoolean(HoodieLockConfig.EARLY_CONFLICT_DETECTION_ENABLE); + return getBoolean(EARLY_CONFLICT_DETECTION_ENABLE); } public String getEarlyConflictDetectionStrategyClassName() { - return getString(HoodieLockConfig.EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME); - } - - public HoodieEarlyConflictDetectionStrategy getEarlyConflictDetectionStrategy() { - return ReflectionUtils.loadClass(getString(HoodieLockConfig.EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME)); + return getString(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME); } // misc configs @@ -2670,6 +2708,26 @@ public Builder doSkipDefaultPartitionValidation(boolean skipDefaultPartitionVali return this; } + public Builder withEarlyConflictDetectionEnable(boolean enable) { + writeConfig.setValue(EARLY_CONFLICT_DETECTION_ENABLE, String.valueOf(enable)); + return this; + } + + public Builder withMarkerConflictCheckerBatchInterval(long interval) { + writeConfig.setValue(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, String.valueOf(interval)); + return this; + } + + public Builder withMarkerConflictCheckerPeriod(long period) { + writeConfig.setValue(MARKER_CONFLICT_CHECKER_PERIOD, String.valueOf(period)); + return this; + } + + public Builder withEarlyConflictDetectionStrategy(String className) { + writeConfig.setValue(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME, className); + return this; + } + protected void setDefaults() { writeConfig.setDefaultValue(MARKERS_TYPE, getDefaultMarkersType(engineType)); // Check for mandatory properties diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java index 1ad28d14b3a8..fe103055c038 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java @@ -19,6 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; @@ -40,5 +41,5 @@ public abstract class HoodieIOHandle { this.fs = getFileSystem(); } - protected abstract FileSystem getFileSystem(); + public abstract FileSystem getFileSystem(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java index fee75b22decd..7b21045cbb4a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java @@ -45,7 +45,7 @@ public HoodieReadHandle(HoodieWriteConfig config, HoodieTable hoodie } @Override - protected FileSystem getFileSystem() { + public FileSystem getFileSystem() { return hoodieTable.getMetaClient().getFs(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 156a291c77e1..80b25157b7c1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -20,9 +20,6 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.transaction.TransactionManager; -import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; -import org.apache.hudi.common.conflict.detection.HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; @@ -30,8 +27,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -41,7 +38,6 @@ import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.marker.WriteMarkers; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.avro.Schema; @@ -56,8 +52,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; @@ -194,54 +188,8 @@ protected Path makeNewFilePath(String partitionPath, String fileName) { * @param partitionPath Partition path */ protected void createMarkerFile(String partitionPath, String dataFileName) { - WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime); - // do early conflict detection before create markers. - if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() - && config.isEarlyConflictDetectionEnable()) { - HoodieEarlyConflictDetectionStrategy earlyConflictDetectionStrategy = config.getEarlyConflictDetectionStrategy(); - if (earlyConflictDetectionStrategy instanceof HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy) { - createMarkerWithTransaction(earlyConflictDetectionStrategy, writeMarkers, partitionPath, dataFileName); - } else { - createMarkerWithEarlyConflictDetection(earlyConflictDetectionStrategy, writeMarkers, partitionPath, dataFileName); - } - } else { - // create marker directly - writeMarkers.create(partitionPath, dataFileName, getIOType()); - } - } - - private Option createMarkerWithEarlyConflictDetection(HoodieEarlyConflictDetectionStrategy resolutionStrategy, - WriteMarkers writeMarkers, - String partitionPath, - String dataFileName) { - Set completedCommitInstants = hoodieTable.getMetaClient().getActiveTimeline() - .getCommitsTimeline() - .filterCompletedInstants() - .getInstants() - .collect(Collectors.toSet()); - - return writeMarkers.createWithEarlyConflictDetection(partitionPath, dataFileName, getIOType(), false, resolutionStrategy, completedCommitInstants, config, fileId); - - } - - private Option createMarkerWithTransaction(HoodieEarlyConflictDetectionStrategy resolutionStrategy, - WriteMarkers writeMarkers, - String partitionPath, - String dataFileName) { - TransactionManager txnManager = new TransactionManager(config, fs, partitionPath, fileId); - try { - // Need to do transaction before create marker file when using early conflict detection - txnManager.beginTransaction(partitionPath, fileId); - return createMarkerWithEarlyConflictDetection(resolutionStrategy, writeMarkers, partitionPath, dataFileName); - - } catch (Exception e) { - LOG.warn("Exception occurs during create marker file in early conflict detection mode."); - throw e; - } finally { - // End transaction after created marker file. - txnManager.endTransaction(partitionPath + "/" + fileId); - txnManager.close(); - } + WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime) + .create(partitionPath, dataFileName, getIOType(), Option.of(this)); } public Schema getWriterSchemaWithMetaFields() { @@ -309,10 +257,18 @@ public String getPartitionPath() { public abstract IOType getIOType(); @Override - protected FileSystem getFileSystem() { + public FileSystem getFileSystem() { return hoodieTable.getMetaClient().getFs(); } + public HoodieWriteConfig getConfig() { + return this.config; + } + + public HoodieTableMetaClient getHoodieTableMetaClient() { + return hoodieTable.getMetaClient(); + } + protected int getPartitionId() { return taskContextSupplier.getPartitionIdSupplier().get(); } @@ -365,6 +321,10 @@ protected HoodieLogFormat.Writer createLogWriter(String baseCommitTime, String f } } + public String getFileId() { + return this.fileId; + } + private static class IgnoreRecord implements GenericRecord { @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index 139ad3664881..9a880f4d7e7c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -18,19 +18,18 @@ package org.apache.hudi.table.marker; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.model.IOType; 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.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; 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.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -162,20 +161,12 @@ protected Option create(String partitionPath, String dataFileName, IOType } @Override - public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, - HoodieEarlyConflictDetectionStrategy resolutionStrategy, - Set completedCommitInstants, HoodieWriteConfig config, String fileId) { - HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) resolutionStrategy; - HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) - .setProperties(config.getProps()).build(); - - if (strategy.hasMarkerConflict(basePath, fs, partitionPath, fileId, instantTime, completedCommitInstants, metaClient)) { - strategy.resolveMarkerConflict(basePath, partitionPath, fileId); - } + public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, + HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { + + HoodieEarlyConflictDetectionStrategy earlyConflictDetectionStrategy = (HoodieEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), + basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); + earlyConflictDetectionStrategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index e9291845a763..85e6ab1ebf0a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -20,8 +20,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; -import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; @@ -30,19 +32,26 @@ import java.io.IOException; import java.util.ConcurrentModificationException; import java.util.Set; +import java.util.stream.Collectors; /** * This strategy is used for direct marker writers, trying to do early conflict detection. * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. */ public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); + public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); + } + @Override - public boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, - Set completedCommitInstants, HoodieTableMetaClient metaClient) { + protected boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, + Set completedCommitInstants) { try { - return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(metaClient, completedCommitInstants, fileId); + return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(completedCommitInstants, fileId, basePath); } catch (IOException e) { LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); @@ -50,7 +59,20 @@ public boolean hasMarkerConflict(String basePath, FileSystem fs, String partitio } @Override - public void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { + protected void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); } + + @Override + public void detectAndResolveConflictIfNecessary() { + + Set completedCommitInstants = activeTimeline.getCommitsTimeline() + .filterCompletedInstants() + .getInstants() + .collect(Collectors.toSet()); + + if (hasMarkerConflict(basePath, fs, partitionPath, fileId, instantTime, completedCommitInstants)) { + resolveMarkerConflict(basePath, partitionPath, fileId); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index 8da21c6c81fb..81251249a70b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -1,56 +1,58 @@ -/* - * 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.marker; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hudi.common.conflict.detection.HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.util.ConcurrentModificationException; -import java.util.Set; - -/** - * This strategy is used for direct marker writers, trying to do early conflict detection. - * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. - */ -public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); - - @Override - public boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, - Set completedCommitInstants, HoodieTableMetaClient metaClient) { - try { - return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(metaClient, completedCommitInstants, fileId); - } catch (IOException e) { - LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); - throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); - } - } - - @Override - public void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { - throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); - } -} +///* +// * 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.marker; +// +//import org.apache.hadoop.fs.FileSystem; +//import org.apache.hudi.client.transaction.TransactionManager; +//import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +//import org.apache.hudi.config.HoodieWriteConfig; +//import org.apache.log4j.LogManager; +//import org.apache.log4j.Logger; +// +///** +// * This strategy is used for direct marker writers, trying to do early conflict detection. +// * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. +// */ +//public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy extends SimpleDirectMarkerBasedEarlyConflictDetectionStrategy { +// +// private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); +// +// public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, +// HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { +// super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); +// } +// +// @Override +// public void detectAndResolveConflictIfNecessary() { +// TransactionManager txnManager = new TransactionManager((HoodieWriteConfig) config, fs, partitionPath, fileId); +// try { +// // Need to do transaction before create marker file when using early conflict detection +// txnManager.beginTransaction(partitionPath, fileId); +// super.detectAndResolveConflictIfNecessary(); +// +// } catch (Exception e) { +// LOG.warn("Exception occurs during create marker file in early conflict detection mode."); +// throw e; +// } finally { +// // End transaction after created marker file. +// txnManager.endTransaction(partitionPath, fileId); +// txnManager.close(); +// } +// } +//} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index bc75c3316c9c..16818445a4f2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -18,10 +18,10 @@ package org.apache.hudi.table.marker; -import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; @@ -155,9 +155,8 @@ protected Option create(String partitionPath, String dataFileName, IOType } @Override - public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, - HoodieEarlyConflictDetectionStrategy resolutionStrategy, - Set completedCommitInstants, HoodieWriteConfig config, String fileId) { + public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, + HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); Map paramsMap = initConfigMap(partitionPath, markerFileName); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index c4ebcf200905..3178b57b31c5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -18,22 +18,24 @@ package org.apache.hudi.table.marker; -import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.Path; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.HoodieWriteHandle; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.io.Serializable; import java.util.Set; +import java.util.stream.Collectors; /** * Operates on markers for a given write action (commit, delta commit, compaction). @@ -55,16 +57,42 @@ public WriteMarkers(String basePath, String markerFolderPath, String instantTime this.instantTime = instantTime; } + public Option create(String partitionPath, String dataFileName, IOType type) { + return create(partitionPath, dataFileName, type, Option.empty()); + } + /** * Creates a marker without checking if the marker already exists. * * @param partitionPath partition path in the table * @param dataFileName data file name * @param type write IO type + * @param handler could be empty * @return the marker path */ - public Option create(String partitionPath, String dataFileName, IOType type) { - return create(partitionPath, dataFileName, type, false); + public Option create(String partitionPath, String dataFileName, IOType type, Option handler) { + boolean checkIfExists = false; + + if (handler.isPresent() + && handler.get().getConfig().getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && handler.get().getConfig().isEarlyConflictDetectionEnable()) { + + HoodieTableMetaClient metaClient = handler.get().getHoodieTableMetaClient(); + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + + Set completedCommitInstants = metaClient.getActiveTimeline() + .getCommitsTimeline() + .filterCompletedInstants() + .getInstants() + .collect(Collectors.toSet()); + + String fileId = handler.get().getFileId(); + HoodieWriteConfig config = handler.get().getConfig(); + return createWithEarlyConflictDetection(partitionPath, dataFileName, type, checkIfExists, completedCommitInstants, config, fileId, activeTimeline); + } else { + // create marker directly + return create(partitionPath, dataFileName, type, checkIfExists); + } } /** @@ -178,7 +206,6 @@ protected Path getMarkerPath(String partitionPath, String dataFileName, IOType t * @param checkIfExists whether to check if the marker already exists * @return the marker path or empty option if already exists and {@code checkIfExists} is true */ - public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, - HoodieEarlyConflictDetectionStrategy resolutionStrategy, - Set completedCommitInstants, HoodieWriteConfig config, String fileId); + public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, + HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline); } 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 index f37486cdf1c0..bbe92d8ec77d 100644 --- 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 @@ -762,12 +762,11 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withAutoArchive(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withMarkersType(MarkerType.DIRECT.name()) - .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) - .withEarlyConflictDetectionEnable(true) - .withEarlyConflictDetectionStrategy(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()) - .withMarkerConflictCheckerBatchInterval(0) - .withMarkerConflictCheckerPeriod(100) - .build()) + .withEarlyConflictDetectionEnable(true) + .withEarlyConflictDetectionStrategy(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()) + .withMarkerConflictCheckerBatchInterval(0) + .withMarkerConflictCheckerPeriod(100) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) .withAutoCommit(false).withProperties(properties).build(); } else { return getConfigBuilder() @@ -783,12 +782,11 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withAutoArchive(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withMarkersType(MarkerType.TIMELINE_SERVER_BASED.name()) - .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) - .withEarlyConflictDetectionEnable(true) - .withEarlyConflictDetectionStrategy(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()) - .withMarkerConflictCheckerBatchInterval(0) - .withMarkerConflictCheckerPeriod(100) - .build()) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) + .withEarlyConflictDetectionEnable(true) + .withEarlyConflictDetectionStrategy(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()) + .withMarkerConflictCheckerBatchInterval(0) + .withMarkerConflictCheckerPeriod(100) .withAutoCommit(false).withProperties(properties).build(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index dbc0a7148ca8..bbc9854fe940 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -18,11 +18,15 @@ package org.apache.hudi.common.conflict.detection; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; @@ -36,12 +40,31 @@ import java.util.stream.Stream; public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.class); + protected final String basePath; + protected final FileSystem fs; + protected final String partitionPath; + protected final String fileId; + protected final String instantTime; + protected final HoodieActiveTimeline activeTimeline; + protected final HoodieConfig config; + + + public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieConfig config) { + this.basePath = basePath; + this.fs = fs; + this.partitionPath = partitionPath; + this.fileId = fileId; + this.instantTime = instantTime; + this.activeTimeline = activeTimeline; + this.config = config; + } - public abstract boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String dataFileName, String instantTime, - Set completedCommitInstants, HoodieTableMetaClient metaClient); + protected abstract boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String dataFileName, String instantTime, Set completedCommitInstants); - public abstract void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName); + protected abstract void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName); /** * We need to do list operation here. @@ -93,8 +116,12 @@ private String getTempFolderPath(String basePath) { return basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; } - public boolean checkCommitConflict(HoodieTableMetaClient metaClient, Set completedCommitInstants, String fileId) { - Set currentInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + public boolean checkCommitConflict(Set completedCommitInstants, String fileId, String basePath) { + + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath).build(); + HoodieActiveTimeline currentActiveTimeline = metaClient.getActiveTimeline().reload(); + + Set currentInstants = currentActiveTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); currentInstants.removeAll(completedCommitInstants); Set missingFileIDs = currentInstants.stream().flatMap(instant -> { try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java index 9c8be817aaaa..6e00b9961504 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java @@ -19,4 +19,6 @@ package org.apache.hudi.common.conflict.detection; public interface HoodieEarlyConflictDetectionStrategy { + + void detectAndResolveConflictIfNecessary(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java index 08a119817f88..b11c680d3e90 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java @@ -25,9 +25,19 @@ public abstract class HoodieTimelineServerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { - public abstract boolean hasMarkerConflict(); + protected final String basePath; + protected final String markerDir; + protected final String markerName; - public abstract void resolveMarkerConflict(String basePath, String markerDir, String markerName); + public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName) { + this.basePath = basePath; + this.markerDir = markerDir; + this.markerName = markerName; + } + + protected abstract boolean hasMarkerConflict(); + + protected abstract void resolveMarkerConflict(String basePath, String markerDir, String markerName); public void fresh(String batchInterval, String period, String markerDir, String basePath, String maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) {} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java deleted file mode 100644 index 39db22687714..000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.common.conflict.detection; - -public abstract class HoodieTransactionDirectMarkerBasedEarlyConflictDetectionStrategy - extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { -} diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index bfb36fcd77d2..92587ded9f80 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -177,7 +177,7 @@ public CompletableFuture createMarker(Context context, String markerDir, try { synchronized (earlyConflictDetectionLock) { if (earlyConflictDetectionStrategy == null) { - earlyConflictDetectionStrategy = ReflectionUtils.loadClass(earlyConflictDetectionClassName); + earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(earlyConflictDetectionClassName, basePath, markerDir, markerName); } if (!markerDir.equalsIgnoreCase(currentMarkerDir)) { @@ -195,9 +195,8 @@ public CompletableFuture createMarker(Context context, String markerDir, } } - if (earlyConflictDetectionStrategy.hasMarkerConflict()) { - earlyConflictDetectionStrategy.resolveMarkerConflict(basePath, markerDir, markerName); - } + earlyConflictDetectionStrategy.detectAndResolveConflictIfNecessary(); + } catch (Exception ex) { LOG.warn("Failed to create marker with early conflict detection enable", ex); MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index b0b1d412f55b..60523890486b 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -34,10 +34,16 @@ import java.util.concurrent.atomic.AtomicBoolean; public class AsyncTimelineMarkerEarlyConflictDetectionStrategy extends HoodieTimelineServerBasedEarlyConflictDetectionStrategy { + private static final Logger LOG = LogManager.getLogger(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class); + private AtomicBoolean hasConflict = new AtomicBoolean(false); private ScheduledExecutorService markerChecker; + public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName) { + super(basePath, markerDir, markerName); + } + @Override public boolean hasMarkerConflict() { return hasConflict.get(); @@ -58,4 +64,11 @@ public void fresh(String batchInterval, String period, String markerDir, String markerChecker.scheduleAtFixedRate(new MarkerCheckerRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants), Long.parseLong(batchInterval), Long.parseLong(period), TimeUnit.MILLISECONDS); } + + @Override + public void detectAndResolveConflictIfNecessary() { + if (hasMarkerConflict()) { + resolveMarkerConflict(basePath, markerDir, markerName); + } + } } From 7d8f3bcf641ff5a3d4525ee17103f384bb985a34 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 09:33:33 +0800 Subject: [PATCH 12/51] refact abstraction --- ...erBasedEarlyConflictDetectionStrategy.java | 16 +-- ...erBasedEarlyConflictDetectionStrategy.java | 116 +++++++++--------- ...erBasedEarlyConflictDetectionStrategy.java | 8 +- .../HoodieEarlyConflictDetectionStrategy.java | 7 ++ ...erBasedEarlyConflictDetectionStrategy.java | 4 - 5 files changed, 72 insertions(+), 79 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index 85e6ab1ebf0a..400e2e1456e4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -31,8 +31,6 @@ import java.io.IOException; import java.util.ConcurrentModificationException; -import java.util.Set; -import java.util.stream.Collectors; /** * This strategy is used for direct marker writers, trying to do early conflict detection. @@ -48,10 +46,9 @@ public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho } @Override - protected boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, - Set completedCommitInstants) { + public boolean hasMarkerConflict() { try { - return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(completedCommitInstants, fileId, basePath); + return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(fileId, basePath); } catch (IOException e) { LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); @@ -59,19 +56,14 @@ protected boolean hasMarkerConflict(String basePath, FileSystem fs, String parti } @Override - protected void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { + public void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName) { throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); } @Override public void detectAndResolveConflictIfNecessary() { - Set completedCommitInstants = activeTimeline.getCommitsTimeline() - .filterCompletedInstants() - .getInstants() - .collect(Collectors.toSet()); - - if (hasMarkerConflict(basePath, fs, partitionPath, fileId, instantTime, completedCommitInstants)) { + if (hasMarkerConflict()) { resolveMarkerConflict(basePath, partitionPath, fileId); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index 81251249a70b..ef22c4a45cf6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -1,58 +1,58 @@ -///* -// * 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.marker; -// -//import org.apache.hadoop.fs.FileSystem; -//import org.apache.hudi.client.transaction.TransactionManager; -//import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -//import org.apache.hudi.config.HoodieWriteConfig; -//import org.apache.log4j.LogManager; -//import org.apache.log4j.Logger; -// -///** -// * This strategy is used for direct marker writers, trying to do early conflict detection. -// * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. -// */ -//public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy extends SimpleDirectMarkerBasedEarlyConflictDetectionStrategy { -// -// private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); -// -// public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, FileSystem fs, String partitionPath, String fileId, String instantTime, -// HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { -// super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); -// } -// -// @Override -// public void detectAndResolveConflictIfNecessary() { -// TransactionManager txnManager = new TransactionManager((HoodieWriteConfig) config, fs, partitionPath, fileId); -// try { -// // Need to do transaction before create marker file when using early conflict detection -// txnManager.beginTransaction(partitionPath, fileId); -// super.detectAndResolveConflictIfNecessary(); -// -// } catch (Exception e) { -// LOG.warn("Exception occurs during create marker file in early conflict detection mode."); -// throw e; -// } finally { -// // End transaction after created marker file. -// txnManager.endTransaction(partitionPath, fileId); -// txnManager.close(); -// } -// } -//} +/* + * 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.marker; + +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * This strategy is used for direct marker writers, trying to do early conflict detection. + * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. + */ +public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy extends SimpleDirectMarkerBasedEarlyConflictDetectionStrategy { + + private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); + + public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); + } + + @Override + public void detectAndResolveConflictIfNecessary() { + TransactionManager txnManager = new TransactionManager((HoodieWriteConfig) config, fs, partitionPath, fileId); + try { + // Need to do transaction before create marker file when using early conflict detection + txnManager.beginTransaction(partitionPath, fileId); + super.detectAndResolveConflictIfNecessary(); + + } catch (Exception e) { + LOG.warn("Exception occurs during create marker file in early conflict detection mode."); + throw e; + } finally { + // End transaction after created marker file. + txnManager.endTransaction(partitionPath, fileId); + txnManager.close(); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index bbc9854fe940..2fbc57e66858 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -49,6 +49,7 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final String instantTime; protected final HoodieActiveTimeline activeTimeline; protected final HoodieConfig config; + private final Set completedCommitInstants; public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, @@ -60,12 +61,9 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho this.instantTime = instantTime; this.activeTimeline = activeTimeline; this.config = config; + this.completedCommitInstants= activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); } - protected abstract boolean hasMarkerConflict(String basePath, FileSystem fs, String partitionPath, String dataFileName, String instantTime, Set completedCommitInstants); - - protected abstract void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName); - /** * We need to do list operation here. * In order to reduce the list pressure as much as possible, first we build path prefix in advance: '$base_path/.temp/instant_time/partition_path', @@ -116,7 +114,7 @@ private String getTempFolderPath(String basePath) { return basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; } - public boolean checkCommitConflict(Set completedCommitInstants, String fileId, String basePath) { + public boolean checkCommitConflict(String fileId, String basePath) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath).build(); HoodieActiveTimeline currentActiveTimeline = metaClient.getActiveTimeline().reload(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java index 6e00b9961504..c32fabe00a76 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java @@ -18,7 +18,14 @@ package org.apache.hudi.common.conflict.detection; +/** + * + */ public interface HoodieEarlyConflictDetectionStrategy { void detectAndResolveConflictIfNecessary(); + + boolean hasMarkerConflict(); + + void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java index b11c680d3e90..0017d9422818 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java @@ -35,10 +35,6 @@ public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, this.markerName = markerName; } - protected abstract boolean hasMarkerConflict(); - - protected abstract void resolveMarkerConflict(String basePath, String markerDir, String markerName); - public void fresh(String batchInterval, String period, String markerDir, String basePath, String maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) {} } From fc5927ad1f2f6688c44bbc2fd4413a0c4c454086 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 11:56:18 +0800 Subject: [PATCH 13/51] address comments --- .../TimelineServerBasedWriteMarkers.java | 25 ++++++------ ...erBasedEarlyConflictDetectionStrategy.java | 8 +--- .../common/table/marker/MarkerOperation.java | 12 +++--- .../apache/hudi/common/util/MarkerUtils.java | 38 ------------------- .../service/handlers/MarkerHandler.java | 6 +-- .../marker/MarkerCheckerRunnable.java | 5 ++- 6 files changed, 30 insertions(+), 64 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 16818445a4f2..62d1298c34f4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -143,7 +143,7 @@ protected Option create(String partitionPath, String dataFileName, IOType HoodieTimer timer = HoodieTimer.start(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = initConfigMap(partitionPath, markerFileName); + Map paramsMap = initConfigMap(partitionPath, markerFileName, Option.empty(), false); boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName + " in " + timer.endTimer() + " ms"); @@ -159,15 +159,7 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = initConfigMap(partitionPath, markerFileName); - - paramsMap.put(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, config.getMarkerConflictCheckerBatchInterval()); - paramsMap.put(MARKER_CONFLICT_CHECKER_PERIOD, config.getMarkerConflictCheckerPeriod()); - paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); - paramsMap.put(MARKER_BASEPATH_PARAM, basePath); - paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs())); - paramsMap.put(MARKER_CONFLICT_CHECKER_ENABLE, String.valueOf(config.isEarlyConflictDetectionEnable())); - paramsMap.put(MARKER_CONFLICT_CHECKER_STRATEGY, config.getEarlyConflictDetectionStrategyClassName()); + Map paramsMap = initConfigMap(partitionPath, markerFileName, Option.of(config), true); boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); @@ -207,7 +199,7 @@ private boolean executeCreateMarkerRequest(Map paramsMap, String * @param markerFileName * @return */ - private Map initConfigMap(String partitionPath, String markerFileName) { + private Map initConfigMap(String partitionPath, String markerFileName, Option hoodieWriteConfig, boolean initEarlyConflictConfigs) { Map paramsMap = new HashMap<>(); paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); @@ -216,6 +208,17 @@ private Map initConfigMap(String partitionPath, String markerFil } else { paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName); } + + if (initEarlyConflictConfigs) { + HoodieWriteConfig config = hoodieWriteConfig.get(); + paramsMap.put(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, config.getMarkerConflictCheckerBatchInterval()); + paramsMap.put(MARKER_CONFLICT_CHECKER_PERIOD, config.getMarkerConflictCheckerPeriod()); + paramsMap.put(MARKER_BASEPATH_PARAM, basePath); + paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs())); + paramsMap.put(MARKER_CONFLICT_CHECKER_ENABLE, String.valueOf(config.isEarlyConflictDetectionEnable())); + paramsMap.put(MARKER_CONFLICT_CHECKER_STRATEGY, config.getEarlyConflictDetectionStrategyClassName()); + } + return paramsMap; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 2fbc57e66858..79169a2be9dd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -76,7 +76,7 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho */ public boolean checkMarkerConflict(String basePath, String partitionPath, String fileId, FileSystem fs, String instantTime) throws IOException { - String tempFolderPath = getTempFolderPath(basePath); + String tempFolderPath = basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; long res = Arrays.stream(fs.listStatus(new Path(tempFolderPath))) .parallel() .map(FileStatus::getPath) @@ -110,11 +110,7 @@ public boolean checkMarkerConflict(String basePath, String partitionPath, String return false; } - private String getTempFolderPath(String basePath) { - return basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; - } - - public boolean checkCommitConflict(String fileId, String basePath) { + protected boolean checkCommitConflict(String fileId, String basePath) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath).build(); HoodieActiveTimeline currentActiveTimeline = metaClient.getActiveTimeline().reload(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java index ba5625480f64..26040491eb93 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java @@ -29,12 +29,7 @@ public class MarkerOperation implements Serializable { public static final String MARKER_DIR_PATH_PARAM = "markerdirpath"; public static final String MARKER_NAME_PARAM = "markername"; - public static final String MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = "batchinterval"; - public static final String MARKER_CONFLICT_CHECKER_PERIOD = "period"; - public static final String MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL = "heartbeatinterval"; public static final String MARKER_BASEPATH_PARAM = "basepath"; - public static final String MARKER_CONFLICT_CHECKER_ENABLE = "HoodieEarlyConflictDetectionStrategy"; - public static final String MARKER_CONFLICT_CHECKER_STRATEGY = "earlyconflictdetectionstrategy"; // GET requests public static final String ALL_MARKERS_URL = String.format("%s/%s", BASE_URL, "all"); @@ -44,4 +39,11 @@ public class MarkerOperation implements Serializable { // POST requests public static final String CREATE_MARKER_URL = String.format("%s/%s", BASE_URL, "create"); public static final String DELETE_MARKER_DIR_URL = String.format("%s/%s", BASE_URL, "dir/delete"); + + // marker conflict check related configs. + public static final String MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = "markerbatchinterval"; + public static final String MARKER_CONFLICT_CHECKER_PERIOD = "markercheckperiod"; + public static final String MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL = "markerheartbeatinterval"; + public static final String MARKER_CONFLICT_CHECKER_ENABLE = "markerconflictenable"; + public static final String MARKER_CONFLICT_CHECKER_STRATEGY = "markerearlyconflictdetectionstrategy"; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 49300206c0e2..ce03ad498347 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -227,44 +227,6 @@ public static Set readMarkersFromFile(Path markersFilePath, Serializable return markers; } - /** - * Reads files containing the markers written by timeline-server-based marker mechanism locally instead of using cluster Context. - * - * @param markerDir marker directory. - * @param fileSystem file system to use. - * @return A {@code Map} of file name to the set of markers stored in the file. - */ - public static Set readTimelineServerBasedMarkersFromFileSystemLocally(String markerDir, FileSystem fileSystem) { - Path dirPath = new Path(markerDir); - try { - if (fileSystem.exists(dirPath)) { - Predicate prefixFilter = fileStatus -> - fileStatus.getPath().getName().startsWith(MARKERS_FILENAME_PREFIX); - Predicate markerTypeFilter = fileStatus -> - !fileStatus.getPath().getName().equals(MARKER_TYPE_FILENAME); - - CopyOnWriteArraySet result = new CopyOnWriteArraySet<>(); - FileStatus[] fileStatuses = fileSystem.listStatus(dirPath); - List subPaths = Arrays.stream(fileStatuses) - .filter(prefixFilter.and(markerTypeFilter)) - .map(fileStatus -> fileStatus.getPath().toString()) - .collect(Collectors.toList()); - - if (subPaths.size() > 0) { - SerializableConfiguration conf = new SerializableConfiguration(fileSystem.getConf()); - subPaths.stream().parallel().forEach(subPath -> { - result.addAll(readMarkersFromFile(new Path(subPath), conf, true)); - }); - } - return result; - } - return new HashSet<>(); - } catch (Exception ioe) { - LOG.warn("IOException occurs during read TimelineServer based markers from fileSystem", ioe); - return new HashSet<>(); - } - } - public static List getAllMarkerDir(Path tempPath, FileSystem fs) throws IOException { return Arrays.stream(fs.listStatus(tempPath)).map(FileStatus::getPath).collect(Collectors.toList()); } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 92587ded9f80..70feaa57fd8d 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -95,7 +95,6 @@ public class MarkerHandler extends Handler { private transient HoodieEngineContext hoodieEngineContext; private ScheduledFuture dispatchingThreadFuture; private boolean firstCreationRequestSeen; - private final ConcurrentHashMap checkers; private String currentMarkerDir = null; private HoodieTimelineServerBasedEarlyConflictDetectionStrategy earlyConflictDetectionStrategy; @@ -114,7 +113,6 @@ public MarkerHandler(Configuration conf, TimelineService.Config timelineServiceC this.markerCreationDispatchingRunnable = new MarkerCreationDispatchingRunnable(markerDirStateMap, batchingExecutorService); this.firstCreationRequestSeen = false; - this.checkers = new ConcurrentHashMap<>(); } /** @@ -126,7 +124,6 @@ public void stop() { } dispatchingExecutorService.shutdown(); batchingExecutorService.shutdown(); - checkers.values().forEach(ExecutorService::shutdown); } /** @@ -180,6 +177,9 @@ public CompletableFuture createMarker(Context context, String markerDir, earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(earlyConflictDetectionClassName, basePath, markerDir, markerName); } + // markerDir => $base_path/.hoodie/.temp/$instant_time + // If markerDir is changed like move to the next instant action, we need to fresh this earlyConflictDetectionStrategy. + // For specific instant related create marker action, we only call this check/fresh once instead of starting the checker for every request if (!markerDir.equalsIgnoreCase(currentMarkerDir)) { this.currentMarkerDir = markerDir; Set actions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java index 05479e73e888..582b36b88a98 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -32,6 +33,7 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -75,7 +77,8 @@ public void run() { List instants = MarkerUtils.getAllMarkerDir(tempPath, fs); List candidate = getCandidateInstants(instants, markerDirToInstantTime(markerDir)); Set tableMarkers = candidate.stream().flatMap(instant -> { - return MarkerUtils.readTimelineServerBasedMarkersFromFileSystemLocally(instant, fs).stream(); + return MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(instant, fs, new HoodieLocalEngineContext(new Configuration()), 100) + .values().stream().flatMap(Collection::stream); }).collect(Collectors.toSet()); Set currentFileIDs = currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); From 3bde14b3e752d56344cba40fbfadbe51a8030ab1 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 18:58:07 +0800 Subject: [PATCH 14/51] address comments --- ...erBasedEarlyConflictDetectionStrategy.java | 2 - ...erBasedEarlyConflictDetectionStrategy.java | 2 +- hudi-client/hudi-spark-client/pom.xml | 7 + .../client/TestHoodieClientMultiWriter.java | 60 +++++++-- .../marker/MarkerCheckerRunnable.java | 3 +- .../marker/TestMarkerCheckerRunnable.java | 125 ++++++++++++++++++ 6 files changed, 183 insertions(+), 16 deletions(-) create mode 100644 hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index 400e2e1456e4..e5585b0bf6bb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -18,11 +18,9 @@ package org.apache.hudi.table.marker; -import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieIOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index ef22c4a45cf6..b79a52a68b92 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -47,7 +47,7 @@ public void detectAndResolveConflictIfNecessary() { super.detectAndResolveConflictIfNecessary(); } catch (Exception e) { - LOG.warn("Exception occurs during create marker file in early conflict detection mode."); + LOG.warn("Exception occurs during create marker file in early conflict detection mode within transaction."); throw e; } finally { // End transaction after created marker file. diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index de51e4480761..5075bc687367 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -215,6 +215,13 @@ awaitility test + + + org.apache.curator + curator-test + ${zk-curator.version} + test + 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 index bbe92d8ec77d..ab20a4e794de 100644 --- 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 @@ -18,8 +18,10 @@ package org.apache.hudi.client; +import org.apache.curator.test.TestingServer; import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; @@ -44,6 +46,7 @@ import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.table.marker.SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hadoop.fs.Path; @@ -82,6 +85,11 @@ import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY; import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY; import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP_KEY; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -155,15 +163,33 @@ private static Iterable providerClassAndTableType() { */ @ParameterizedTest @MethodSource("configParams") - public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String tableType, String markerType) throws Exception { + public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String tableType, String markerType, String earlyConflictDetectionStrategy) throws Exception { if (tableType.equalsIgnoreCase(HoodieTableType.MERGE_ON_READ.name())) { setUpMORTestTable(); } - Properties properties = new Properties(); - properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); - properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); - HoodieWriteConfig writeConfig = buildWriteConfigForEarlyConflictDetect(markerType, properties); + HoodieWriteConfig writeConfig; + TestingServer server = null; + if (earlyConflictDetectionStrategy.equalsIgnoreCase(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName())) { + // need to setup zk related env there. Bcz SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy is only support zk lock for now. + server = new TestingServer(); + Properties properties = new Properties(); + properties.setProperty(ZK_BASE_PATH_PROP_KEY, basePath); + properties.setProperty(ZK_CONNECT_URL_PROP_KEY, server.getConnectString()); + properties.setProperty(ZK_BASE_PATH_PROP_KEY, server.getTempDirectory().getAbsolutePath()); + properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP_KEY, "10000"); + properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY, "10000"); + properties.setProperty(ZK_LOCK_KEY_PROP_KEY, "key"); + properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000"); + + writeConfig = buildWriteConfigForEarlyConflictDetect(markerType, properties, ZookeeperBasedLockProvider.class, earlyConflictDetectionStrategy); + } else { + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); + writeConfig = buildWriteConfigForEarlyConflictDetect(markerType, properties, InProcessLockProvider.class, earlyConflictDetectionStrategy); + } + // Create the first commit final String nextCommitTime1 = "001"; createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", nextCommitTime1, 2000, true); @@ -196,6 +222,9 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta assertTrue(completedInstant.contains(nextCommitTime1)); assertTrue(completedInstant.contains(nextCommitTime2)); FileIOUtils.deleteDirectory(new File(basePath)); + if (server != null) { + server.close(); + } } @ParameterizedTest @@ -743,12 +772,19 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, public static Stream configParams() { Object[][] data = - new Object[][] {{"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name()}, {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name()}}; + new Object[][] { + {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} + }; return Stream.of(data).map(Arguments::of); } - private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerType, Properties properties) { + private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerType, Properties properties, + Class lockProvider, String earlyConflictDetectionStrategy) { if (markerType.equalsIgnoreCase(MarkerType.DIRECT.name())) { return getConfigBuilder() .withHeartbeatIntervalInMs(3600 * 1000) @@ -763,10 +799,10 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withMarkersType(MarkerType.DIRECT.name()) .withEarlyConflictDetectionEnable(true) - .withEarlyConflictDetectionStrategy(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()) + .withEarlyConflictDetectionStrategy(earlyConflictDetectionStrategy) .withMarkerConflictCheckerBatchInterval(0) .withMarkerConflictCheckerPeriod(100) - .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(lockProvider).build()) .withAutoCommit(false).withProperties(properties).build(); } else { return getConfigBuilder() @@ -782,9 +818,9 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withAutoArchive(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withMarkersType(MarkerType.TIMELINE_SERVER_BASED.name()) - .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(lockProvider).build()) .withEarlyConflictDetectionEnable(true) - .withEarlyConflictDetectionStrategy(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()) + .withEarlyConflictDetectionStrategy(earlyConflictDetectionStrategy) .withMarkerConflictCheckerBatchInterval(0) .withMarkerConflictCheckerPeriod(100) .withAutoCommit(false).withProperties(properties).build(); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java index 582b36b88a98..cb96f3d4a5b9 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -70,7 +71,7 @@ public void run() { return; } - HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieTimer timer = HoodieTimer.start(); Set currentInstantAllMarkers = markerHandler.getAllMarkers(markerDir); Path tempPath = new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME); diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java new file mode 100644 index 000000000000..6a2805319b29 --- /dev/null +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java @@ -0,0 +1,125 @@ +/* + * 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.timeline.service.handlers.marker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.timeline.service.handlers.MarkerHandler; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.BufferedWriter; +import java.io.FileWriter; +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestMarkerCheckerRunnable extends HoodieCommonTestHarness { + + private static final Logger LOG = LogManager.getLogger(TestMarkerCheckerRunnable.class); + + @BeforeEach + public void setUp() throws Exception { + initPath(); + } + + @AfterEach + public void tearDown() throws Exception { + Path path = new Path(basePath); + FileSystem fs = path.getFileSystem(new Configuration()); + fs.delete(path, true); + } + + /** + * Prepare dataset : + * $base_path/.hoodie/.temp/2016/001/MARKERS0 => 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0_4-17-21_001.parquet.marker.CREATE (same) + * 2016/4a266542-c7d5-426f-8fb8-fb85a2e88448-0_3-17-20_001.parquet.marker.CREATE + * /002/MARKERS0 => 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0_40-170-210_002.parquet.marker.MERGE (same) + * => 2016/1228caeb-4188-4e19-a18d-848e6f9b0448-0_55-55-425_002.parquet.marker.MERGE + * + * + * Run MarkerCheckerRunnable and find there is a conflict 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0 + */ + @Test + public void testMarkerCheckerRunnable() throws IOException, InterruptedException { + + AtomicBoolean hasConflict = new AtomicBoolean(false); + FileSystem fs = new Path(basePath).getFileSystem(new Configuration()); + MarkerHandler markerHandler = mock(MarkerHandler.class); + String rootBaseMarkerDir = basePath + "/.hoodie/.temp"; + String partition = "2016"; + + String oldInstant = "001"; + Set oldMarkers = Stream.of(partition + "/b21adfa2-7013-4452-a565-4cc39fea5b73-0_4-17-21_001.parquet.marker.CREATE", + partition + "/4a266542-c7d5-426f-8fb8-fb85a2e88448-0_3-17-20_001.parquet.marker.CREATE").collect(Collectors.toSet()); + prepareFiles(rootBaseMarkerDir, oldInstant, oldMarkers, fs); + + // here current markers and old markers have a common fileID b21adfa2-7013-4452-a565-4cc39fea5b73-0 + String currentInstantTime = "002"; + String currentMarkerDir = rootBaseMarkerDir + "/" + currentInstantTime; + Set currentMarkers = Stream.of(partition + "/b21adfa2-7013-4452-a565-4cc39fea5b73-0_40-170-210_002.parquet.marker.MERGE", + partition + "/1228caeb-4188-4e19-a18d-848e6f9b0448-0_55-55-425_002.parquet.marker.MERGE").collect(Collectors.toSet()); + prepareFiles(rootBaseMarkerDir, currentInstantTime, currentMarkers, fs); + + HashSet oldInstants = new HashSet<>(); + oldInstants.add(new HoodieInstant(false, "commit", oldInstant)); + when(markerHandler.getAllMarkers(currentMarkerDir)).thenReturn(currentMarkers); + + ScheduledExecutorService markerChecker = Executors.newSingleThreadScheduledExecutor(); + markerChecker.submit(new MarkerCheckerRunnable(hasConflict, markerHandler, currentMarkerDir, + basePath, fs, Long.MAX_VALUE, oldInstants)); + + markerChecker.shutdown(); + markerChecker.awaitTermination(60, TimeUnit.SECONDS); + + assertTrue(hasConflict.get()); + } + + private void prepareFiles(String baseMarkerDir, String instant, Set markers, FileSystem fs) throws IOException { + String markerDir = baseMarkerDir + "/" + instant; + fs.mkdirs(new Path(markerDir)); + BufferedWriter out = new BufferedWriter(new FileWriter(markerDir + "/MARKERS0")); + markers.forEach(ele -> { + try { + out.write(ele); + out.write("\n"); + } catch (IOException e) { + // ignore here. + } + }); + + out.close(); + } +} From ea2719e75553dfdcd1aa200e1a1f56392f3ffec1 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 19:04:13 +0800 Subject: [PATCH 15/51] address comments --- ...rectMarkerBasedEarlyConflictDetectionStrategy.java | 3 +-- .../java/org/apache/hudi/common/util/MarkerUtils.java | 1 - ...cTimelineMarkerEarlyConflictDetectionStrategy.java | 2 +- ...=> MarkerBasedEarlyConflictDetectionRunnable.java} | 11 +++++------ ...estMarkerBasedEarlyConflictDetectionRunnable.java} | 6 +++--- 5 files changed, 10 insertions(+), 13 deletions(-) rename hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/{MarkerCheckerRunnable.java => MarkerBasedEarlyConflictDetectionRunnable.java} (93%) rename hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/{TestMarkerCheckerRunnable.java => TestMarkerBasedEarlyConflictDetectionRunnable.java} (95%) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 79169a2be9dd..825be7695aec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -51,7 +51,6 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final HoodieConfig config; private final Set completedCommitInstants; - public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, HoodieActiveTimeline activeTimeline, HoodieConfig config) { this.basePath = basePath; @@ -61,7 +60,7 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho this.instantTime = instantTime; this.activeTimeline = activeTimeline; this.config = config; - this.completedCommitInstants= activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index ce03ad498347..d2cd116198bc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -45,7 +45,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CopyOnWriteArraySet; import java.util.function.Predicate; import java.util.stream.Collectors; diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index 60523890486b..945553ec0d9d 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -61,7 +61,7 @@ public void fresh(String batchInterval, String period, String markerDir, String } hasConflict.compareAndSet(true, false); markerChecker = Executors.newSingleThreadScheduledExecutor(); - markerChecker.scheduleAtFixedRate(new MarkerCheckerRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, + markerChecker.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants), Long.parseLong(batchInterval), Long.parseLong(period), TimeUnit.MILLISECONDS); } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java similarity index 93% rename from hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java rename to hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index cb96f3d4a5b9..a58528517641 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -35,14 +35,13 @@ import java.io.IOException; import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; -public class MarkerCheckerRunnable implements Runnable { - private static final Logger LOG = LogManager.getLogger(MarkerCheckerRunnable.class); +public class MarkerBasedEarlyConflictDetectionRunnable implements Runnable { + private static final Logger LOG = LogManager.getLogger(MarkerBasedEarlyConflictDetectionRunnable.class); private MarkerHandler markerHandler; private String markerDir; @@ -52,9 +51,9 @@ public class MarkerCheckerRunnable implements Runnable { private long maxAllowableHeartbeatIntervalInMs; private Set oldInstants; - public MarkerCheckerRunnable(AtomicBoolean hasConflict, MarkerHandler markerHandler, String markerDir, - String basePath, FileSystem fileSystem, long maxAllowableHeartbeatIntervalInMs, - Set oldInstants) { + public MarkerBasedEarlyConflictDetectionRunnable(AtomicBoolean hasConflict, MarkerHandler markerHandler, String markerDir, + String basePath, FileSystem fileSystem, long maxAllowableHeartbeatIntervalInMs, + Set oldInstants) { this.markerHandler = markerHandler; this.markerDir = markerDir; this.basePath = basePath; diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java similarity index 95% rename from hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java rename to hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java index 6a2805319b29..c5008c0f4d73 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerCheckerRunnable.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java @@ -46,9 +46,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class TestMarkerCheckerRunnable extends HoodieCommonTestHarness { +public class TestMarkerBasedEarlyConflictDetectionRunnable extends HoodieCommonTestHarness { - private static final Logger LOG = LogManager.getLogger(TestMarkerCheckerRunnable.class); + private static final Logger LOG = LogManager.getLogger(TestMarkerBasedEarlyConflictDetectionRunnable.class); @BeforeEach public void setUp() throws Exception { @@ -98,7 +98,7 @@ public void testMarkerCheckerRunnable() throws IOException, InterruptedException when(markerHandler.getAllMarkers(currentMarkerDir)).thenReturn(currentMarkers); ScheduledExecutorService markerChecker = Executors.newSingleThreadScheduledExecutor(); - markerChecker.submit(new MarkerCheckerRunnable(hasConflict, markerHandler, currentMarkerDir, + markerChecker.submit(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, markerHandler, currentMarkerDir, basePath, fs, Long.MAX_VALUE, oldInstants)); markerChecker.shutdown(); From 844b10a447845b27056b2520b67a624a0d1bd71a Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 19:09:46 +0800 Subject: [PATCH 16/51] address comments --- .../org/apache/hudi/client/transaction/lock/LockManager.java | 4 ++-- .../main/java/org/apache/hudi/config/HoodieWriteConfig.java | 1 - .../src/main/java/org/apache/hudi/io/HoodieIOHandle.java | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) 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 index 30cebf8647d6..acf89aeae49a 100644 --- 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 @@ -88,8 +88,8 @@ private void init(HoodieWriteConfig writeConfig, Configuration conf) { */ private TypedProperties refreshLockConfig(HoodieWriteConfig writeConfig, String key) { TypedProperties props = new TypedProperties(writeConfig.getProps()); - String zk_base_path = props.getProperty(LockConfiguration.ZK_BASE_PATH_PROP_KEY); - if (StringUtils.isNullOrEmpty(zk_base_path)) { + String zkBasePath = props.getProperty(LockConfiguration.ZK_BASE_PATH_PROP_KEY); + if (StringUtils.isNullOrEmpty(zkBasePath)) { throw new HoodieNotSupportedException("Only Support ZK based lock for now."); } props.setProperty(LockConfiguration.ZK_LOCK_KEY_PROP_KEY, key); 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 6698c49377e3..481b3bf34d44 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 @@ -30,7 +30,6 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieMetastoreConfig; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FileSystemRetryConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java index fe103055c038..743d28ee5f41 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java @@ -19,7 +19,6 @@ package org.apache.hudi.io; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; From 71e0d1e91d2f540b441838263362c31251fa4ab5 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 19:12:15 +0800 Subject: [PATCH 17/51] address comments --- .../org/apache/hudi/client/TestHoodieClientMultiWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index ab20a4e794de..2572d2d6c975 100644 --- 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 @@ -779,7 +779,7 @@ public static Stream configParams() { {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} - }; + }; return Stream.of(data).map(Arguments::of); } From 374212b88a0be4364320a049f0f257f5993d0587 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 23:30:57 +0800 Subject: [PATCH 18/51] neeed to fix reflection issue --- .../apache/hudi/config/HoodieWriteConfig.java | 15 ++++++++ .../hudi/table/marker/DirectWriteMarkers.java | 10 +++--- ...erBasedEarlyConflictDetectionStrategy.java | 11 ++++-- ...erBasedEarlyConflictDetectionStrategy.java | 4 +-- .../TimelineServerBasedWriteMarkers.java | 2 ++ .../client/TestHoodieClientMultiWriter.java | 10 +++--- ...erBasedEarlyConflictDetectionStrategy.java | 26 +++----------- .../HoodieEarlyConflictDetectionStrategy.java | 2 +- ...erBasedEarlyConflictDetectionStrategy.java | 4 ++- .../common/table/marker/MarkerOperation.java | 1 + .../apache/hudi/common/util/MarkerUtils.java | 25 +++++++++++++ .../hudi/timeline/service/RequestHandler.java | 3 +- .../service/handlers/MarkerHandler.java | 7 ++-- ...eMarkerEarlyConflictDetectionStrategy.java | 7 ++-- ...erBasedEarlyConflictDetectionRunnable.java | 35 ++++--------------- ...erBasedEarlyConflictDetectionRunnable.java | 2 +- 16 files changed, 91 insertions(+), 73 deletions(-) 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 481b3bf34d44..51af255f6282 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 @@ -553,6 +553,12 @@ public class HoodieWriteConfig extends HoodieConfig { .sinceVersion("0.13.0") .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking."); + public static final ConfigProperty MARKER_CONFLICT_CHECK_COMMIT_CONFLICT = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.check.commit.conflict") + .defaultValue(false) + .sinceVersion("0.13.0") + .withDocumentation("Enable check commit conflict or not during early conflict detect"); + private ConsistencyGuardConfig consistencyGuardConfig; private FileSystemRetryConfig fileSystemRetryConfig; @@ -2195,6 +2201,10 @@ public String getEarlyConflictDetectionStrategyClassName() { return getString(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME); } + public boolean checkCommitConflictDuringEarlyConflictDetect() { + return getBoolean(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT); + } + // misc configs public Boolean doSkipDefaultPartitionValidation() { return getBoolean(SKIP_DEFAULT_PARTITION_VALIDATION); @@ -2722,6 +2732,11 @@ public Builder withMarkerConflictCheckerPeriod(long period) { return this; } + public Builder withCheckCommitConflict(boolean enable) { + writeConfig.setValue(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.valueOf(enable)); + return this; + } + public Builder withEarlyConflictDetectionStrategy(String className) { writeConfig.setValue(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME, className); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index 9a880f4d7e7c..bdc998f877c5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.marker; import org.apache.hudi.common.config.SerializableConfiguration; -import org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; @@ -164,9 +164,11 @@ protected Option create(String partitionPath, String dataFileName, IOType public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { - HoodieEarlyConflictDetectionStrategy earlyConflictDetectionStrategy = (HoodieEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), - basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); - earlyConflictDetectionStrategy.detectAndResolveConflictIfNecessary(); + HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), + basePath, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect()); + + + strategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index e5585b0bf6bb..080d3e21f3db 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -18,9 +18,11 @@ package org.apache.hudi.table.marker; +import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieIOException; @@ -29,6 +31,8 @@ import java.io.IOException; import java.util.ConcurrentModificationException; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * This strategy is used for direct marker writers, trying to do early conflict detection. @@ -39,14 +43,15 @@ public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends Hoodi private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, boolean checkCommitConflict) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict); } @Override public boolean hasMarkerConflict() { try { - return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) || checkCommitConflict(fileId, basePath); + return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) + || (checkCommitConflict && MarkerUtils.hasCommitConflict(Stream.of(fileId).collect(Collectors.toSet()), basePath, completedCommitInstants)); } catch (IOException e) { LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index b79a52a68b92..bf30ba91f4f5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -34,8 +34,8 @@ public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy ex private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config); + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, boolean checkCommitConflict) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 62d1298c34f4..74738ef09104 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -59,6 +59,7 @@ import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_PERIOD; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY; +import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECK_COMMIT_CONFLICT; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_DIR_PATH_PARAM; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_NAME_PARAM; @@ -217,6 +218,7 @@ private Map initConfigMap(String partitionPath, String markerFil paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs())); paramsMap.put(MARKER_CONFLICT_CHECKER_ENABLE, String.valueOf(config.isEarlyConflictDetectionEnable())); paramsMap.put(MARKER_CONFLICT_CHECKER_STRATEGY, config.getEarlyConflictDetectionStrategyClassName()); + paramsMap.put(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.valueOf(config.checkCommitConflictDuringEarlyConflictDetect())); } return paramsMap; 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 index 2572d2d6c975..c08d621a5dd0 100644 --- 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 @@ -773,12 +773,12 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, public static Stream configParams() { Object[][] data = new Object[][] { - {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, +// {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, +// {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, +// {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, - {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} +// {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, +// {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} }; return Stream.of(data).map(Arguments::of); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 825be7695aec..24833438b47a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -18,13 +18,11 @@ package org.apache.hudi.common.conflict.detection; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -49,10 +47,11 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final String instantTime; protected final HoodieActiveTimeline activeTimeline; protected final HoodieConfig config; - private final Set completedCommitInstants; + protected final Set completedCommitInstants; + protected final boolean checkCommitConflict; public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieConfig config) { + HoodieActiveTimeline activeTimeline, HoodieConfig config, boolean checkCommitConflict) { this.basePath = basePath; this.fs = fs; this.partitionPath = partitionPath; @@ -61,6 +60,7 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho this.activeTimeline = activeTimeline; this.config = config; this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + this.checkCommitConflict = checkCommitConflict; } /** @@ -108,22 +108,4 @@ public boolean checkMarkerConflict(String basePath, String partitionPath, String } return false; } - - protected boolean checkCommitConflict(String fileId, String basePath) { - - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath).build(); - HoodieActiveTimeline currentActiveTimeline = metaClient.getActiveTimeline().reload(); - - Set currentInstants = currentActiveTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); - currentInstants.removeAll(completedCommitInstants); - Set missingFileIDs = currentInstants.stream().flatMap(instant -> { - try { - return HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class) - .getFileIdAndRelativePaths().keySet().stream(); - } catch (Exception e) { - return Stream.empty(); - } - }).collect(Collectors.toSet()); - return missingFileIDs.contains(fileId); - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java index c32fabe00a76..8bbeef146ee8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java @@ -21,7 +21,7 @@ /** * */ -public interface HoodieEarlyConflictDetectionStrategy { +interface HoodieEarlyConflictDetectionStrategy { void detectAndResolveConflictIfNecessary(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java index 0017d9422818..9806c982d793 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java @@ -28,11 +28,13 @@ public abstract class HoodieTimelineServerBasedEarlyConflictDetectionStrategy im protected final String basePath; protected final String markerDir; protected final String markerName; + protected final boolean checkCommitConflict; - public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName) { + public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, boolean checkCommitConflict) { this.basePath = basePath; this.markerDir = markerDir; this.markerName = markerName; + this.checkCommitConflict = checkCommitConflict; } public void fresh(String batchInterval, String period, String markerDir, String basePath, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java index 26040491eb93..10f2990d1e7b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java @@ -46,4 +46,5 @@ public class MarkerOperation implements Serializable { public static final String MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL = "markerheartbeatinterval"; public static final String MARKER_CONFLICT_CHECKER_ENABLE = "markerconflictenable"; public static final String MARKER_CONFLICT_CHECKER_STRATEGY = "markerearlyconflictdetectionstrategy"; + public static final String MARKER_CONFLICT_CHECK_COMMIT_CONFLICT = "markercheckcommitconflict"; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index d2cd116198bc..ff61630b4dc1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -19,11 +19,14 @@ package org.apache.hudi.common.util; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -47,6 +50,7 @@ import java.util.Set; import java.util.function.Predicate; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.util.FileIOUtils.closeQuietly; @@ -229,4 +233,25 @@ public static Set readMarkersFromFile(Path markersFilePath, Serializable public static List getAllMarkerDir(Path tempPath, FileSystem fs) throws IOException { return Arrays.stream(fs.listStatus(tempPath)).map(FileStatus::getPath).collect(Collectors.toList()); } + + public static boolean hasCommitConflict(Set currentFileIDs, String basePath, Set completedCommitInstants) { + + HoodieTableMetaClient metaClient = + HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true).build(); + + Set currentInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + + currentInstants.removeAll(completedCommitInstants); + Set missingFileIDs = currentInstants.stream().flatMap(instant -> { + try { + return HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class) + .getFileIdAndRelativePaths().keySet().stream(); + } catch (Exception e) { + return Stream.empty(); + } + }).collect(Collectors.toSet()); + currentFileIDs.retainAll(missingFileIDs); + return !currentFileIDs.isEmpty(); + } } \ No newline at end of file diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index ceb09873138d..d146df1bc474 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -466,7 +466,8 @@ private void registerMarkerAPI() { ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.class).getOrDefault(""), ctx.queryParamAsClass(MarkerOperation.MARKER_BASEPATH_PARAM, String.class).getOrDefault(""), ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_ENABLE, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY, String.class).getOrDefault(""))); + ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY, String.class).getOrDefault(""), + ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.class).getOrDefault(""))); }, false)); app.post(MarkerOperation.DELETE_MARKER_DIR_URL, new ViewHandler(ctx -> { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 70feaa57fd8d..3f66a26e58d8 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.sun.org.apache.xpath.internal.operations.Bool; import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; @@ -168,13 +169,15 @@ public boolean doesMarkerDirExist(String markerDir) { public CompletableFuture createMarker(Context context, String markerDir, String markerName, String batchInterval, String period, String maxAllowableHeartbeatIntervalInMs, String basePath, String earlyConflictDetectionEnable, - String earlyConflictDetectionClassName) { + String earlyConflictDetectionClassName, + String checkCommitConflict) { // Step1 do early conflict detection if enable if (Boolean.parseBoolean(earlyConflictDetectionEnable)) { try { synchronized (earlyConflictDetectionLock) { if (earlyConflictDetectionStrategy == null) { - earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(earlyConflictDetectionClassName, basePath, markerDir, markerName); + earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(earlyConflictDetectionClassName, + basePath, markerDir, markerName, Boolean.parseBoolean(checkCommitConflict)); } // markerDir => $base_path/.hoodie/.temp/$instant_time diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index 945553ec0d9d..3506ad35fdec 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -40,8 +40,8 @@ public class AsyncTimelineMarkerEarlyConflictDetectionStrategy extends HoodieTim private AtomicBoolean hasConflict = new AtomicBoolean(false); private ScheduledExecutorService markerChecker; - public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName) { - super(basePath, markerDir, markerName); + public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, boolean checkCommitConflict) { + super(basePath, markerDir, markerName, checkCommitConflict); } @Override @@ -62,7 +62,8 @@ public void fresh(String batchInterval, String period, String markerDir, String hasConflict.compareAndSet(true, false); markerChecker = Executors.newSingleThreadScheduledExecutor(); markerChecker.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, - fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants), Long.parseLong(batchInterval), Long.parseLong(period), TimeUnit.MILLISECONDS); + fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants, checkCommitConflict), + Long.parseLong(batchInterval), Long.parseLong(period), TimeUnit.MILLISECONDS); } @Override diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index a58528517641..9ea7f8384711 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -22,7 +22,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; @@ -38,7 +37,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -import java.util.stream.Stream; public class MarkerBasedEarlyConflictDetectionRunnable implements Runnable { private static final Logger LOG = LogManager.getLogger(MarkerBasedEarlyConflictDetectionRunnable.class); @@ -50,10 +48,11 @@ public class MarkerBasedEarlyConflictDetectionRunnable implements Runnable { private AtomicBoolean hasConflict; private long maxAllowableHeartbeatIntervalInMs; private Set oldInstants; + private final boolean checkCommitConflict; public MarkerBasedEarlyConflictDetectionRunnable(AtomicBoolean hasConflict, MarkerHandler markerHandler, String markerDir, String basePath, FileSystem fileSystem, long maxAllowableHeartbeatIntervalInMs, - Set oldInstants) { + Set oldInstants, boolean checkCommitConflict) { this.markerHandler = markerHandler; this.markerDir = markerDir; this.basePath = basePath; @@ -61,6 +60,7 @@ public MarkerBasedEarlyConflictDetectionRunnable(AtomicBoolean hasConflict, Mark this.hasConflict = hasConflict; this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; this.oldInstants = oldInstants; + this.checkCommitConflict = checkCommitConflict; } @Override @@ -86,7 +86,8 @@ public void run() { currentFileIDs.retainAll(tableFilesIDs); - if (!currentFileIDs.isEmpty() || hasCommitConflict(currentInstantAllMarkers, basePath)) { + if (!currentFileIDs.isEmpty() + || (checkCommitConflict && MarkerUtils.hasCommitConflict(currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()), basePath, oldInstants))) { LOG.warn("Conflict writing detected based on markers!\n" + "Conflict markers: " + currentInstantAllMarkers + "\n" + "Table markers: " + tableMarkers); @@ -152,7 +153,7 @@ private static String markerDirToInstantTime(String marker) { * @return * @throws IOException */ - public Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { + private Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); if (fs.exists(heartbeatFilePath)) { return fs.getFileStatus(heartbeatFilePath).getModificationTime(); @@ -162,7 +163,7 @@ public Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantT } } - public boolean isHeartbeatExpired(String instantTime) throws IOException { + private boolean isHeartbeatExpired(String instantTime) throws IOException { Long currentTime = System.currentTimeMillis(); Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); if (currentTime - lastHeartbeatTime > this.maxAllowableHeartbeatIntervalInMs) { @@ -171,26 +172,4 @@ public boolean isHeartbeatExpired(String instantTime) throws IOException { } return false; } - - public boolean hasCommitConflict(Set currentInstantAllMarkers, String basePath) { - Set currentFileIDs = currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); - - HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath) - .setLoadActiveTimelineOnLoad(true).build(); - - Set currentInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); - - currentInstants.removeAll(oldInstants); - Set missingFileIDs = currentInstants.stream().flatMap(instant -> { - try { - return HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class) - .getFileIdAndRelativePaths().keySet().stream(); - } catch (Exception e) { - return Stream.empty(); - } - }).collect(Collectors.toSet()); - currentFileIDs.retainAll(missingFileIDs); - return !currentFileIDs.isEmpty(); - } } diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java index c5008c0f4d73..5da169ae0f70 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java @@ -99,7 +99,7 @@ public void testMarkerCheckerRunnable() throws IOException, InterruptedException ScheduledExecutorService markerChecker = Executors.newSingleThreadScheduledExecutor(); markerChecker.submit(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, markerHandler, currentMarkerDir, - basePath, fs, Long.MAX_VALUE, oldInstants)); + basePath, fs, Long.MAX_VALUE, oldInstants, true)); markerChecker.shutdown(); markerChecker.awaitTermination(60, TimeUnit.SECONDS); From 8dfdb4a9de4344dd5746bda3cecd651c9856de3d Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 23:52:16 +0800 Subject: [PATCH 19/51] address comments --- .../apache/hudi/table/marker/DirectWriteMarkers.java | 3 +-- ...irectMarkerBasedEarlyConflictDetectionStrategy.java | 3 +-- ...irectMarkerBasedEarlyConflictDetectionStrategy.java | 2 +- .../hudi/client/TestHoodieClientMultiWriter.java | 10 +++++----- ...irectMarkerBasedEarlyConflictDetectionStrategy.java | 2 +- ...elineServerBasedEarlyConflictDetectionStrategy.java | 2 +- ...ncTimelineMarkerEarlyConflictDetectionStrategy.java | 2 +- 7 files changed, 11 insertions(+), 13 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index bdc998f877c5..edb52bc56dcd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -165,8 +165,7 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), - basePath, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect()); - + basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect()); strategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index 080d3e21f3db..d35c78b1a6ae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.marker; -import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -43,7 +42,7 @@ public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends Hoodi private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, boolean checkCommitConflict) { + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict) { super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index bf30ba91f4f5..a8d4d43f0ae2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -34,7 +34,7 @@ public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy ex private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, boolean checkCommitConflict) { + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict) { super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict); } 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 index c08d621a5dd0..2572d2d6c975 100644 --- 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 @@ -773,12 +773,12 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, public static Stream configParams() { Object[][] data = new Object[][] { -// {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, -// {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, -// {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, -// {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, -// {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} + {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} }; return Stream.of(data).map(Arguments::of); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 24833438b47a..6def6d9c895c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -51,7 +51,7 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final boolean checkCommitConflict; public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieConfig config, boolean checkCommitConflict) { + HoodieActiveTimeline activeTimeline, HoodieConfig config, Boolean checkCommitConflict) { this.basePath = basePath; this.fs = fs; this.partitionPath = partitionPath; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java index 9806c982d793..6816868c6e9d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java @@ -30,7 +30,7 @@ public abstract class HoodieTimelineServerBasedEarlyConflictDetectionStrategy im protected final String markerName; protected final boolean checkCommitConflict; - public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, boolean checkCommitConflict) { + public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { this.basePath = basePath; this.markerDir = markerDir; this.markerName = markerName; diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index 3506ad35fdec..61d73b39b3d6 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -40,7 +40,7 @@ public class AsyncTimelineMarkerEarlyConflictDetectionStrategy extends HoodieTim private AtomicBoolean hasConflict = new AtomicBoolean(false); private ScheduledExecutorService markerChecker; - public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, boolean checkCommitConflict) { + public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { super(basePath, markerDir, markerName, checkCommitConflict); } From 6fc5bf1ce7921bf25acc3659565457264d8b9dc2 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 21 Nov 2022 23:57:09 +0800 Subject: [PATCH 20/51] address comments --- .../apache/hudi/client/transaction/lock/LockManager.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) 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 index acf89aeae49a..ca84b7531598 100644 --- 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 @@ -56,8 +56,7 @@ public class LockManager implements Serializable, AutoCloseable { private volatile LockProvider lockProvider; public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { - this.lockConfiguration = new LockConfiguration(writeConfig.getProps()); - init(writeConfig, fs.getConf()); + init(writeConfig, fs.getConf(), writeConfig.getProps()); } /** @@ -69,11 +68,11 @@ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { */ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, String partitionPath, String fileId) { TypedProperties props = refreshLockConfig(writeConfig, partitionPath + "/" + fileId); - this.lockConfiguration = new LockConfiguration(props); - init(writeConfig, fs.getConf()); + init(writeConfig, fs.getConf(), props); } - private void init(HoodieWriteConfig writeConfig, Configuration conf) { + private void init(HoodieWriteConfig writeConfig, Configuration conf, TypedProperties lockProps) { + this.lockConfiguration = new LockConfiguration(lockProps); this.writeConfig = writeConfig; this.hadoopConf = new SerializableConfiguration(conf); this.maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, From 316e5aefa89ec2dc35928af7c7da4b11a23aab20 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 00:38:58 +0800 Subject: [PATCH 21/51] address comments --- .../org/apache/hudi/timeline/service/handlers/MarkerHandler.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 3f66a26e58d8..334b8f68b24a 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.sun.org.apache.xpath.internal.operations.Bool; import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; From 0b74647767677a4cc1193295b493dc0537dd4c96 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 00:54:56 +0800 Subject: [PATCH 22/51] address comments --- ...sactionDirectMarkerBasedEarlyConflictDetectionStrategy.java | 3 +++ .../HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index a8d4d43f0ae2..afcf3edcfe71 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -25,6 +25,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.stream.Collectors; + /** * This strategy is used for direct marker writers, trying to do early conflict detection. * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. @@ -44,6 +46,7 @@ public void detectAndResolveConflictIfNecessary() { try { // Need to do transaction before create marker file when using early conflict detection txnManager.beginTransaction(partitionPath, fileId); + this.completedCommitInstants = activeTimeline.reload().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); super.detectAndResolveConflictIfNecessary(); } catch (Exception e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 6def6d9c895c..c40e037e55cf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -47,7 +47,7 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final String instantTime; protected final HoodieActiveTimeline activeTimeline; protected final HoodieConfig config; - protected final Set completedCommitInstants; + protected Set completedCommitInstants; protected final boolean checkCommitConflict; public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, From c3403d74c6e5fcf0fe9680bb3b78441f3de7a506 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 00:56:53 +0800 Subject: [PATCH 23/51] address comments --- ...ansactionDirectMarkerBasedEarlyConflictDetectionStrategy.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index afcf3edcfe71..5e9be54816c6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -46,7 +46,6 @@ public void detectAndResolveConflictIfNecessary() { try { // Need to do transaction before create marker file when using early conflict detection txnManager.beginTransaction(partitionPath, fileId); - this.completedCommitInstants = activeTimeline.reload().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); super.detectAndResolveConflictIfNecessary(); } catch (Exception e) { From 345a9dfc06b633a931e6440abadf3d730ede2950 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 00:58:49 +0800 Subject: [PATCH 24/51] address comments --- ...nsactionDirectMarkerBasedEarlyConflictDetectionStrategy.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index 5e9be54816c6..a8d4d43f0ae2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -25,8 +25,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.util.stream.Collectors; - /** * This strategy is used for direct marker writers, trying to do early conflict detection. * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. From ffd8315ca6dc2c771e90b5fa983c0b41010605eb Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 00:59:54 +0800 Subject: [PATCH 25/51] address comments --- .../java/org/apache/hudi/table/marker/DirectWriteMarkers.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index edb52bc56dcd..f197d1810163 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -164,7 +164,8 @@ protected Option create(String partitionPath, String dataFileName, IOType public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { - HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), + HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = + (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect()); strategy.detectAndResolveConflictIfNecessary(); From 6ec57feb130747b952661c4c7b989131ab19510c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 01:34:50 +0800 Subject: [PATCH 26/51] address comments --- .../AsyncTimelineMarkerEarlyConflictDetectionStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index 61d73b39b3d6..a71d49a41769 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -59,7 +59,7 @@ public void fresh(String batchInterval, String period, String markerDir, String if (markerChecker != null) { markerChecker.shutdown(); } - hasConflict.compareAndSet(true, false); + hasConflict.set(false); markerChecker = Executors.newSingleThreadScheduledExecutor(); markerChecker.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants, checkCommitConflict), From 1455ab1ea53dc07cc28f12e71b68254c8a82657c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 18:57:44 +0800 Subject: [PATCH 27/51] address comments --- .../hudi/table/marker/DirectWriteMarkers.java | 4 +- ...erBasedEarlyConflictDetectionStrategy.java | 8 +- ...erBasedEarlyConflictDetectionStrategy.java | 4 +- .../TimelineServerBasedWriteMarkers.java | 2 +- .../hudi/table/marker/WriteMarkers.java | 15 ++- .../client/TestHoodieClientMultiWriter.java | 56 ++++++++--- ...erBasedEarlyConflictDetectionStrategy.java | 26 +++--- .../apache/hudi/common/util/MarkerUtils.java | 92 +++++++++++++++++-- ...erBasedEarlyConflictDetectionRunnable.java | 89 +++--------------- 9 files changed, 174 insertions(+), 122 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index f197d1810163..f890a2ab4723 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -164,9 +164,11 @@ protected Option create(String partitionPath, String dataFileName, IOType public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { + long maxAllowableHeartbeatIntervalInMs = config.getHoodieClientHeartbeatIntervalInMs() * config.getHoodieClientHeartbeatTolerableMisses(); + HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), - basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect()); + basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect(), maxAllowableHeartbeatIntervalInMs); strategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index d35c78b1a6ae..fb526a748c6f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -42,15 +42,15 @@ public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends Hoodi private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict); + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs); } @Override public boolean hasMarkerConflict() { try { - return checkMarkerConflict(basePath, partitionPath, fileId, fs, instantTime) - || (checkCommitConflict && MarkerUtils.hasCommitConflict(Stream.of(fileId).collect(Collectors.toSet()), basePath, completedCommitInstants)); + return checkMarkerConflict(activeTimeline, basePath, partitionPath, fileId, fs, instantTime) + || (checkCommitConflict && MarkerUtils.hasCommitConflict(activeTimeline, Stream.of(fileId).collect(Collectors.toSet()), completedCommitInstants)); } catch (IOException e) { LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); throw new HoodieIOException("Exception occurs during create marker file in eager conflict detection mode.", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index a8d4d43f0ae2..c09ccd46f036 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -34,8 +34,8 @@ public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy ex private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict); + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 74738ef09104..f8e6651a581a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -215,7 +215,7 @@ private Map initConfigMap(String partitionPath, String markerFil paramsMap.put(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, config.getMarkerConflictCheckerBatchInterval()); paramsMap.put(MARKER_CONFLICT_CHECKER_PERIOD, config.getMarkerConflictCheckerPeriod()); paramsMap.put(MARKER_BASEPATH_PARAM, basePath); - paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs())); + paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs() * config.getHoodieClientHeartbeatTolerableMisses())); paramsMap.put(MARKER_CONFLICT_CHECKER_ENABLE, String.valueOf(config.isEarlyConflictDetectionEnable())); paramsMap.put(MARKER_CONFLICT_CHECKER_STRATEGY, config.getEarlyConflictDetectionStrategyClassName()); paramsMap.put(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.valueOf(config.checkCommitConflictDuringEarlyConflictDetect())); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 3178b57b31c5..99d17d3cfdc8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.Path; @@ -80,11 +81,15 @@ public Option create(String partitionPath, String dataFileName, IOType typ HoodieTableMetaClient metaClient = handler.get().getHoodieTableMetaClient(); HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - Set completedCommitInstants = metaClient.getActiveTimeline() - .getCommitsTimeline() - .filterCompletedInstants() - .getInstants() - .collect(Collectors.toSet()); + HoodieTimeline pendingCompactionTimeline = activeTimeline.filterPendingCompactionTimeline(); + HoodieTimeline pendingReplaceTimeline = activeTimeline.filterPendingReplaceTimeline(); + // TODO if current is compact or clustering then create marker directly without early conflict detection. + // Need to support early conflict detection between table service and common writers. + if (pendingCompactionTimeline.containsInstant(instantTime) || pendingReplaceTimeline.containsInstant(instantTime)) { + return create(partitionPath, dataFileName, type, checkIfExists); + } + + Set completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); String fileId = handler.get().getFileId(); HoodieWriteConfig config = handler.get().getConfig(); 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 index 2572d2d6c975..290491b095ed 100644 --- 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 @@ -29,12 +29,14 @@ 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.HoodieTableMetaClient; import org.apache.hudi.common.table.marker.MarkerType; 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.FileIOUtils; +import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCleanConfig; @@ -151,12 +153,14 @@ private static Iterable providerClassAndTableType() { * 3. MOR + Timeline server based marker * 4. COW + Timeline server based marker * - * ---|---------|--------------------|--------------------------------------|-------------------------> time + * |---------------------- 003 heartBeat expired -------------------| + * + * ---|---------|--------------------|--------------------------------------|-------------------------|-------------------------> time * init 001 * 002 start writing * 003 start which has conflict with 002 * and failed soon - * 002 commit successfully + * 002 commit successfully 004 write successfully * @param tableType * @param markerType * @throws Exception @@ -168,6 +172,8 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta setUpMORTestTable(); } + int heartBeatIntervalForCommit4 = 10 * 1000; + HoodieWriteConfig writeConfig; TestingServer server = null; if (earlyConflictDetectionStrategy.equalsIgnoreCase(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName())) { @@ -192,7 +198,7 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta // Create the first commit final String nextCommitTime1 = "001"; - createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", nextCommitTime1, 2000, true); + createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", nextCommitTime1, 200, true); final SparkRDDWriteClient client2 = getHoodieWriteClient(writeConfig); final SparkRDDWriteClient client3 = getHoodieWriteClient(writeConfig); @@ -200,13 +206,13 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta final String nextCommitTime2 = "002"; // start to write commit 002 - final JavaRDD writeStatusList2 = startCommitForUpdate(writeConfig, client2, nextCommitTime2, 1000); + final JavaRDD writeStatusList2 = startCommitForUpdate(writeConfig, client2, nextCommitTime2, 100); // start to write commit 003 // this commit 003 will failed quickly because early conflict detection before create marker. final String nextCommitTime3 = "003"; assertThrows(SparkException.class, () -> { - final JavaRDD writeStatusList3 = startCommitForUpdate(writeConfig, client3, nextCommitTime3, 1000); + final JavaRDD writeStatusList3 = startCommitForUpdate(writeConfig, client3, nextCommitTime3, 100); client3.commit(nextCommitTime3, writeStatusList3); }, "Early conflict detected but cannot resolve conflicts for overlapping writes"); @@ -215,12 +221,30 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta client2.commit(nextCommitTime2, writeStatusList2); }); + HoodieWriteConfig config4 = HoodieWriteConfig.newBuilder().withProperties(writeConfig.getProps()).withHeartbeatIntervalInMs(heartBeatIntervalForCommit4).build(); + final SparkRDDWriteClient client4 = getHoodieWriteClient(config4); + + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + nextCommitTime3); + fs.create(heartbeatFilePath, true); + + // Wait for heart beat expired for failed commitTime3 "003" + // Otherwise commit4 still can see conflict between failed write 003. + Thread.sleep(heartBeatIntervalForCommit4 * 2); + + final String nextCommitTime4 = "004"; + assertDoesNotThrow(() -> { + final JavaRDD writeStatusList4 = startCommitForUpdate(writeConfig, client4, nextCommitTime4, 100); + client4.commit(nextCommitTime4, writeStatusList4); + }); + List completedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline() .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - assertEquals(2, completedInstant.size()); + assertEquals(3, completedInstant.size()); assertTrue(completedInstant.contains(nextCommitTime1)); assertTrue(completedInstant.contains(nextCommitTime2)); + assertTrue(completedInstant.contains(nextCommitTime4)); + FileIOUtils.deleteDirectory(new File(basePath)); if (server != null) { server.close(); @@ -732,13 +756,22 @@ private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient } } - private void createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommit, - String commitTimeBetweenPrevAndNew, String newCommitTime, int numRecords) + private JavaRDD createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommit, + String commitTimeBetweenPrevAndNew, String newCommitTime, int numRecords) throws Exception { + return createCommitWithUpserts(cfg, client, prevCommit, commitTimeBetweenPrevAndNew, newCommitTime, numRecords, true); + } + + private JavaRDD createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommit, + String commitTimeBetweenPrevAndNew, String newCommitTime, int numRecords, boolean commit) 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); + if (commit) { + client.commit(newCommitTime, result); + } + + return result; } /** @@ -777,7 +810,6 @@ public static Stream configParams() { {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} }; return Stream.of(data).map(Arguments::of); @@ -787,7 +819,7 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy Class lockProvider, String earlyConflictDetectionStrategy) { if (markerType.equalsIgnoreCase(MarkerType.DIRECT.name())) { return getConfigBuilder() - .withHeartbeatIntervalInMs(3600 * 1000) + .withHeartbeatIntervalInMs(60 * 1000) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withStorageType(FileSystemViewStorageType.MEMORY) .withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build()) @@ -807,7 +839,7 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy } else { return getConfigBuilder() .withStorageConfig(HoodieStorageConfig.newBuilder().parquetMaxFileSize(20 * 1024).build()) - .withHeartbeatIntervalInMs(3600 * 1000) + .withHeartbeatIntervalInMs(60 * 1000) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withStorageType(FileSystemViewStorageType.MEMORY) .withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build()) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index c40e037e55cf..8375ad2cdf20 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; @@ -33,6 +34,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -48,19 +50,21 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final HoodieActiveTimeline activeTimeline; protected final HoodieConfig config; protected Set completedCommitInstants; - protected final boolean checkCommitConflict; + protected final Boolean checkCommitConflict; + protected final Long maxAllowableHeartbeatIntervalInMs; public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieConfig config, Boolean checkCommitConflict) { + HoodieActiveTimeline activeTimeline, HoodieConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { this.basePath = basePath; this.fs = fs; this.partitionPath = partitionPath; this.fileId = fileId; this.instantTime = instantTime; + this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); this.activeTimeline = activeTimeline; this.config = config; - this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); this.checkCommitConflict = checkCommitConflict; + this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; } /** @@ -73,20 +77,18 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho * @return true if current fileID is already existed under .temp/instant_time/partition_path/.. * @throws IOException */ - public boolean checkMarkerConflict(String basePath, String partitionPath, String fileId, + public boolean checkMarkerConflict(HoodieActiveTimeline activeTimeline, String basePath, String partitionPath, String fileId, FileSystem fs, String instantTime) throws IOException { String tempFolderPath = basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; - long res = Arrays.stream(fs.listStatus(new Path(tempFolderPath))) - .parallel() - .map(FileStatus::getPath) - .filter(markerPath -> { - return !markerPath.getName().equalsIgnoreCase(instantTime); - }) - .flatMap(currentMarkerDirPath -> { + + List candidateInstants = MarkerUtils.getCandidateInstants(activeTimeline, Arrays.stream(fs.listStatus(new Path(tempFolderPath))).map(FileStatus::getPath).collect(Collectors.toList()), + instantTime, maxAllowableHeartbeatIntervalInMs, fs, basePath); + + long res = candidateInstants.stream().flatMap(currentMarkerDirPath -> { try { Path markerPartitionPath; if (StringUtils.isNullOrEmpty(partitionPath)) { - markerPartitionPath = currentMarkerDirPath; + markerPartitionPath = new Path(currentMarkerDirPath); } else { markerPartitionPath = new Path(currentMarkerDirPath, partitionPath); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index ff61630b4dc1..2c61fa52993d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -26,7 +26,9 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.marker.MarkerType; +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.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -234,18 +236,14 @@ public static List getAllMarkerDir(Path tempPath, FileSystem fs) throws IO return Arrays.stream(fs.listStatus(tempPath)).map(FileStatus::getPath).collect(Collectors.toList()); } - public static boolean hasCommitConflict(Set currentFileIDs, String basePath, Set completedCommitInstants) { + public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set currentFileIDs, Set completedCommitInstants) { - HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath) - .setLoadActiveTimelineOnLoad(true).build(); - - Set currentInstants = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + Set currentInstants = activeTimeline.reload().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); currentInstants.removeAll(completedCommitInstants); Set missingFileIDs = currentInstants.stream().flatMap(instant -> { try { - return HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class) + return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class) .getFileIdAndRelativePaths().keySet().stream(); } catch (Exception e) { return Stream.empty(); @@ -254,4 +252,84 @@ public static boolean hasCommitConflict(Set currentFileIDs, String baseP currentFileIDs.retainAll(missingFileIDs); return !currentFileIDs.isEmpty(); } + + /** + * Get Candidate Instant to do conflict checking: + * 1. Skip current writer related instant(currentInstantTime) + * 2. Skip all instants after currentInstantTime + * 3. Skip dead writers related instants based on heart-beat + * 4. Skip pending compaction instant (For now we don' do early conflict check with compact action) + * Because we don't want to let pending compaction block common writer. + * @param instants + * @return + */ + public static List getCandidateInstants(HoodieActiveTimeline activeTimeline, List instants, String currentInstantTime, + long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) { + + HoodieTimeline pendingCompactionTimeline = activeTimeline.reload().filterPendingCompactionTimeline(); + + return instants.stream().map(Path::toString).filter(instantPath -> { + String instantTime = markerDirToInstantTime(instantPath); + boolean isPendingCompaction = pendingCompactionTimeline.containsInstant(instantTime); + return instantTime.compareToIgnoreCase(currentInstantTime) < 0 && !isPendingCompaction; + }).filter(instantPath -> { + try { + return !isHeartbeatExpired(markerDirToInstantTime(instantPath), maxAllowableHeartbeatIntervalInMs, fs, basePath); + } catch (IOException e) { + return false; + } + }).collect(Collectors.toList()); + } + + /** + * Get fileID from full marker path, for example: + * 20210623/0/20210825/932a86d9-5c1d-44c7-ac99-cb88b8ef8478-0_85-15-1390_20220620181735781.parquet.marker.MERGE + * ==> get 20210623/0/20210825/932a86d9-5c1d-44c7-ac99-cb88b8ef8478-0 + * @param marker + * @return + */ + public static String makerToPartitionAndFileID(String marker) { + String[] ele = marker.split("_"); + return ele[0]; + } + + /** + * Get instantTime from full marker path, for example: + * /var/folders/t3/th1dw75d0yz2x2k2qt6ys9zh0000gp/T/junit6502909693741900820/dataset/.hoodie/.temp/003 + * ==> 003 + * @param marker + * @return + */ + public static String markerDirToInstantTime(String marker) { + String[] ele = marker.split("/"); + return ele[ele.length - 1]; + } + + /** + * Use modification time as last heart beat time + * @param fs + * @param basePath + * @param instantTime + * @return + * @throws IOException + */ + public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); + if (fs.exists(heartbeatFilePath)) { + return fs.getFileStatus(heartbeatFilePath).getModificationTime(); + } else { + // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed + return 0L; + } + } + + public static boolean isHeartbeatExpired(String instantTime, long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) throws IOException { + Long currentTime = System.currentTimeMillis(); + Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); + if (currentTime - lastHeartbeatTime > maxAllowableHeartbeatIntervalInMs) { + LOG.warn("Heartbeat expired, for instant: " + instantTime); + return true; + } + return false; + } } \ No newline at end of file diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index 9ea7f8384711..0a6889e56d88 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -75,19 +75,25 @@ public void run() { Path tempPath = new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME); List instants = MarkerUtils.getAllMarkerDir(tempPath, fs); - List candidate = getCandidateInstants(instants, markerDirToInstantTime(markerDir)); + + HoodieTableMetaClient metaClient = + HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true).build(); + + List candidate = MarkerUtils.getCandidateInstants(metaClient.getActiveTimeline(), instants, + MarkerUtils.markerDirToInstantTime(markerDir), maxAllowableHeartbeatIntervalInMs, fs, basePath); Set tableMarkers = candidate.stream().flatMap(instant -> { return MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(instant, fs, new HoodieLocalEngineContext(new Configuration()), 100) .values().stream().flatMap(Collection::stream); }).collect(Collectors.toSet()); - Set currentFileIDs = currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); - Set tableFilesIDs = tableMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()); + Set currentFileIDs = currentInstantAllMarkers.stream().map(MarkerUtils::makerToPartitionAndFileID).collect(Collectors.toSet()); + Set tableFilesIDs = tableMarkers.stream().map(MarkerUtils::makerToPartitionAndFileID).collect(Collectors.toSet()); currentFileIDs.retainAll(tableFilesIDs); - if (!currentFileIDs.isEmpty() - || (checkCommitConflict && MarkerUtils.hasCommitConflict(currentInstantAllMarkers.stream().map(this::makerToPartitionAndFileID).collect(Collectors.toSet()), basePath, oldInstants))) { + || (checkCommitConflict && MarkerUtils.hasCommitConflict(metaClient.getActiveTimeline(), + currentInstantAllMarkers.stream().map(MarkerUtils::makerToPartitionAndFileID).collect(Collectors.toSet()), oldInstants))) { LOG.warn("Conflict writing detected based on markers!\n" + "Conflict markers: " + currentInstantAllMarkers + "\n" + "Table markers: " + tableMarkers); @@ -99,77 +105,4 @@ public void run() { throw new HoodieIOException("IOException occurs during checking marker conflict"); } } - - /** - * Get Candidate Instant to do conflict checking: - * 1. Skip current writer related instant(currentInstantTime) - * 2. Skip all instants after currentInstantTime - * 3. Skip dead writers related instants based on heart-beat - * @param instants - * @return - */ - private List getCandidateInstants(List instants, String currentInstantTime) { - return instants.stream().map(Path::toString).filter(instantPath -> { - String instantTime = markerDirToInstantTime(instantPath); - return instantTime.compareToIgnoreCase(currentInstantTime) < 0; - }).filter(instantPath -> { - try { - return !isHeartbeatExpired(markerDirToInstantTime(instantPath)); - } catch (IOException e) { - return false; - } - }).collect(Collectors.toList()); - } - - /** - * Get fileID from full marker path, for example: - * 20210623/0/20210825/932a86d9-5c1d-44c7-ac99-cb88b8ef8478-0_85-15-1390_20220620181735781.parquet.marker.MERGE - * ==> get 20210623/0/20210825/932a86d9-5c1d-44c7-ac99-cb88b8ef8478-0 - * @param marker - * @return - */ - private String makerToPartitionAndFileID(String marker) { - String[] ele = marker.split("_"); - return ele[0]; - } - - /** - * Get instantTime from full marker path, for example: - * /var/folders/t3/th1dw75d0yz2x2k2qt6ys9zh0000gp/T/junit6502909693741900820/dataset/.hoodie/.temp/003 - * ==> 003 - * @param marker - * @return - */ - private static String markerDirToInstantTime(String marker) { - String[] ele = marker.split("/"); - return ele[ele.length - 1]; - } - - /** - * Use modification time as last heart beat time - * @param fs - * @param basePath - * @param instantTime - * @return - * @throws IOException - */ - private Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { - Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); - if (fs.exists(heartbeatFilePath)) { - return fs.getFileStatus(heartbeatFilePath).getModificationTime(); - } else { - // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed - return 0L; - } - } - - private boolean isHeartbeatExpired(String instantTime) throws IOException { - Long currentTime = System.currentTimeMillis(); - Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); - if (currentTime - lastHeartbeatTime > this.maxAllowableHeartbeatIntervalInMs) { - LOG.warn("Heartbeat expired, for instant: " + instantTime); - return true; - } - return false; - } } From e13ebb96865c5b96b027121da464e0837aeb635c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 19:00:21 +0800 Subject: [PATCH 28/51] address comments --- ...erBasedEarlyConflictDetectionStrategy.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 8375ad2cdf20..d05caf901ba3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -85,24 +85,24 @@ public boolean checkMarkerConflict(HoodieActiveTimeline activeTimeline, String b instantTime, maxAllowableHeartbeatIntervalInMs, fs, basePath); long res = candidateInstants.stream().flatMap(currentMarkerDirPath -> { - try { - Path markerPartitionPath; - if (StringUtils.isNullOrEmpty(partitionPath)) { - markerPartitionPath = new Path(currentMarkerDirPath); - } else { - markerPartitionPath = new Path(currentMarkerDirPath, partitionPath); - } + try { + Path markerPartitionPath; + if (StringUtils.isNullOrEmpty(partitionPath)) { + markerPartitionPath = new Path(currentMarkerDirPath); + } else { + markerPartitionPath = new Path(currentMarkerDirPath, partitionPath); + } - if (!StringUtils.isNullOrEmpty(partitionPath) && !fs.exists(markerPartitionPath)) { - return Stream.empty(); - } else { - return Arrays.stream(fs.listStatus(markerPartitionPath)).parallel() - .filter((path) -> path.toString().contains(fileId)); - } - } catch (IOException e) { - throw new HoodieIOException("IOException occurs during checking marker file conflict"); - } - }).count(); + if (!StringUtils.isNullOrEmpty(partitionPath) && !fs.exists(markerPartitionPath)) { + return Stream.empty(); + } else { + return Arrays.stream(fs.listStatus(markerPartitionPath)).parallel() + .filter((path) -> path.toString().contains(fileId)); + } + } catch (IOException e) { + throw new HoodieIOException("IOException occurs during checking marker file conflict"); + } + }).count(); if (res != 0L) { LOG.warn("Detected conflict marker files: " + partitionPath + "/" + fileId + " for " + instantTime); From 8a402c4c8574cca66047e614038a002cb24e8e94 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 19:01:53 +0800 Subject: [PATCH 29/51] address comments --- .../src/main/java/org/apache/hudi/common/util/MarkerUtils.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 2c61fa52993d..24408828b17d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -19,7 +19,6 @@ package org.apache.hudi.common.util; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; From a3d0a478937b5ffa0c57caf3a085c0e26ce15f0a Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 19:57:24 +0800 Subject: [PATCH 30/51] address comments --- ...onDirectMarkerBasedEarlyConflictDetectionStrategy.java | 5 +++-- .../java/org/apache/hudi/common/util/MarkerUtils.java | 8 ++++---- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index c09ccd46f036..44c597bba480 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -33,8 +33,9 @@ public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy ex private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); - public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { + public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, + String instantTime, HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, + Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 24408828b17d..d8749fa726ea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.table.marker.MarkerType; 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.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -265,12 +264,13 @@ public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set public static List getCandidateInstants(HoodieActiveTimeline activeTimeline, List instants, String currentInstantTime, long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) { - HoodieTimeline pendingCompactionTimeline = activeTimeline.reload().filterPendingCompactionTimeline(); + HoodieActiveTimeline reloadActive = activeTimeline.reload(); return instants.stream().map(Path::toString).filter(instantPath -> { String instantTime = markerDirToInstantTime(instantPath); - boolean isPendingCompaction = pendingCompactionTimeline.containsInstant(instantTime); - return instantTime.compareToIgnoreCase(currentInstantTime) < 0 && !isPendingCompaction; + return instantTime.compareToIgnoreCase(currentInstantTime) < 0 + && !reloadActive.filterPendingCompactionTimeline() .containsInstant(instantTime) + && !reloadActive.filterPendingReplaceTimeline().containsInstant(instantTime); }).filter(instantPath -> { try { return !isHeartbeatExpired(markerDirToInstantTime(instantPath), maxAllowableHeartbeatIntervalInMs, fs, basePath); From 3369e5e8770cf9eb4c4d272f7c3af54933c992aa Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 19:59:54 +0800 Subject: [PATCH 31/51] address comments --- .../src/main/java/org/apache/hudi/common/util/MarkerUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index d8749fa726ea..7678fc62443f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -269,7 +269,7 @@ public static List getCandidateInstants(HoodieActiveTimeline activeTimel return instants.stream().map(Path::toString).filter(instantPath -> { String instantTime = markerDirToInstantTime(instantPath); return instantTime.compareToIgnoreCase(currentInstantTime) < 0 - && !reloadActive.filterPendingCompactionTimeline() .containsInstant(instantTime) + && !reloadActive.filterPendingCompactionTimeline().containsInstant(instantTime) && !reloadActive.filterPendingReplaceTimeline().containsInstant(instantTime); }).filter(instantPath -> { try { From b97bb164efb5264476ebdf3673396583ec90d1a6 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 20:10:16 +0800 Subject: [PATCH 32/51] address comments --- .../java/org/apache/hudi/client/TestHoodieClientMultiWriter.java | 1 - 1 file changed, 1 deletion(-) 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 index 290491b095ed..029be05c2952 100644 --- 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 @@ -36,7 +36,6 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.FileIOUtils; -import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCleanConfig; From 1ccecb4fa727cc254cf4780012c28bab24e6afde Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 22:26:41 +0800 Subject: [PATCH 33/51] address comments --- .../embedded/EmbeddedTimelineService.java | 12 +++- .../apache/hudi/config/HoodieWriteConfig.java | 10 +-- .../hudi/table/marker/DirectWriteMarkers.java | 2 +- ...erBasedEarlyConflictDetectionStrategy.java | 7 ++- ...erBasedEarlyConflictDetectionStrategy.java | 11 ++-- .../TimelineServerBasedWriteMarkers.java | 19 +----- ...erBasedEarlyConflictDetectionStrategy.java | 6 +- ...erBasedEarlyConflictDetectionStrategy.java | 4 +- .../common/table/marker/MarkerOperation.java | 8 --- .../hudi/timeline/service/RequestHandler.java | 8 +-- .../timeline/service/TimelineService.java | 63 +++++++++++++++++++ .../service/handlers/MarkerHandler.java | 15 ++--- ...eMarkerEarlyConflictDetectionStrategy.java | 8 +-- ...erBasedEarlyConflictDetectionRunnable.java | 6 +- 14 files changed, 116 insertions(+), 63 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index 4d5375894d7e..6b70afcc3c1a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -86,7 +86,17 @@ public void startServer() throws IOException { .enableMarkerRequests(true) .markerBatchNumThreads(writeConfig.getMarkersTimelineServerBasedBatchNumThreads()) .markerBatchIntervalMs(writeConfig.getMarkersTimelineServerBasedBatchIntervalMs()) - .markerParallelism(writeConfig.getMarkersDeleteParallelism()); + .markerParallelism(writeConfig.getMarkersDeleteParallelism()) + ; + } + + if (writeConfig.isEarlyConflictDetectionEnable()) { + timelineServiceConfBuilder.markerEarlyConflictDetectEnable(true) + .markerEarlyConflictDetectStrategy(writeConfig.getEarlyConflictDetectionStrategyClassName()) + .markerEarlyConflictDetectCheckCommitConflict(writeConfig.earlyConflictDetectionCheckCommitConflict()) + .markerEarlyConflictAsyncCheckerBatchInterval(writeConfig.getEarlyConflictDetectionAsyncCheckerBatchInterval()) + .markerEarlyConflictAsyncCheckerBatchPeriod(writeConfig.getEarlyConflictDetectionAsyncCheckerPeriod()) + .markerEarlyConflictMaxAllowableHeartbeatIntervalInMs(writeConfig.getHoodieClientHeartbeatIntervalInMs()); } server = new TimelineService(context, hadoopConf.newCopy(), timelineServiceConfBuilder.build(), 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 51af255f6282..b015218d5c94 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 @@ -2177,12 +2177,12 @@ public ConflictResolutionStrategy getWriteConflictResolutionStrategy() { return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME)); } - public String getMarkerConflictCheckerBatchInterval() { - return String.valueOf(getLong(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL)); + public Long getEarlyConflictDetectionAsyncCheckerBatchInterval() { + return getLong(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL); } - public String getMarkerConflictCheckerPeriod() { - return String.valueOf(getLong(MARKER_CONFLICT_CHECKER_PERIOD)); + public Long getEarlyConflictDetectionAsyncCheckerPeriod() { + return getLong(MARKER_CONFLICT_CHECKER_PERIOD); } public Long getLockAcquireWaitTimeoutInMs() { @@ -2201,7 +2201,7 @@ public String getEarlyConflictDetectionStrategyClassName() { return getString(EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME); } - public boolean checkCommitConflictDuringEarlyConflictDetect() { + public boolean earlyConflictDetectionCheckCommitConflict() { return getBoolean(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index f890a2ab4723..10d013088854 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -168,7 +168,7 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), - basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.checkCommitConflictDuringEarlyConflictDetect(), maxAllowableHeartbeatIntervalInMs); + basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.earlyConflictDetectionCheckCommitConflict(), maxAllowableHeartbeatIntervalInMs, completedCommitInstants); strategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index fb526a748c6f..a850ed2f2d28 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; @@ -30,6 +31,7 @@ import java.io.IOException; import java.util.ConcurrentModificationException; +import java.util.HashSet; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -42,8 +44,9 @@ public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends Hoodi private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs); + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs, + HashSet completedCommitInstants) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs, completedCommitInstants); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index 44c597bba480..bfd60fe3ddac 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -21,10 +21,13 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.HashSet; + /** * This strategy is used for direct marker writers, trying to do early conflict detection. * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. @@ -33,10 +36,10 @@ public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy ex private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); - public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, - String instantTime, HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, - Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs); + public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, + Long maxAllowableHeartbeatIntervalInMs, HashSet completedCommitInstants) { + super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs, completedCommitInstants); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index f8e6651a581a..7fc76d732a0e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -54,12 +54,6 @@ import static org.apache.hudi.common.table.marker.MarkerOperation.DELETE_MARKER_DIR_URL; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKERS_DIR_EXISTS_URL; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_BASEPATH_PARAM; -import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_BATCH_INTERVAL; -import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_ENABLE; -import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL; -import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_PERIOD; -import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY; -import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_CONFLICT_CHECK_COMMIT_CONFLICT; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_DIR_PATH_PARAM; import static org.apache.hudi.common.table.marker.MarkerOperation.MARKER_NAME_PARAM; @@ -144,7 +138,7 @@ protected Option create(String partitionPath, String dataFileName, IOType HoodieTimer timer = HoodieTimer.start(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = initConfigMap(partitionPath, markerFileName, Option.empty(), false); + Map paramsMap = initConfigMap(partitionPath, markerFileName, false); boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName + " in " + timer.endTimer() + " ms"); @@ -160,7 +154,7 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = initConfigMap(partitionPath, markerFileName, Option.of(config), true); + Map paramsMap = initConfigMap(partitionPath, markerFileName, true); boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); @@ -200,7 +194,7 @@ private boolean executeCreateMarkerRequest(Map paramsMap, String * @param markerFileName * @return */ - private Map initConfigMap(String partitionPath, String markerFileName, Option hoodieWriteConfig, boolean initEarlyConflictConfigs) { + private Map initConfigMap(String partitionPath, String markerFileName, boolean initEarlyConflictConfigs) { Map paramsMap = new HashMap<>(); paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); @@ -211,14 +205,7 @@ private Map initConfigMap(String partitionPath, String markerFil } if (initEarlyConflictConfigs) { - HoodieWriteConfig config = hoodieWriteConfig.get(); - paramsMap.put(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, config.getMarkerConflictCheckerBatchInterval()); - paramsMap.put(MARKER_CONFLICT_CHECKER_PERIOD, config.getMarkerConflictCheckerPeriod()); paramsMap.put(MARKER_BASEPATH_PARAM, basePath); - paramsMap.put(MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.valueOf(config.getHoodieClientHeartbeatIntervalInMs() * config.getHoodieClientHeartbeatTolerableMisses())); - paramsMap.put(MARKER_CONFLICT_CHECKER_ENABLE, String.valueOf(config.isEarlyConflictDetectionEnable())); - paramsMap.put(MARKER_CONFLICT_CHECKER_STRATEGY, config.getEarlyConflictDetectionStrategyClassName()); - paramsMap.put(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.valueOf(config.checkCommitConflictDuringEarlyConflictDetect())); } return paramsMap; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index d05caf901ba3..32ccb2051722 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -54,13 +55,14 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final Long maxAllowableHeartbeatIntervalInMs; public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs) { + HoodieActiveTimeline activeTimeline, HoodieConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs, + HashSet completedCommitInstants) { this.basePath = basePath; this.fs = fs; this.partitionPath = partitionPath; this.fileId = fileId; this.instantTime = instantTime; - this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + this.completedCommitInstants = completedCommitInstants; this.activeTimeline = activeTimeline; this.config = config; this.checkCommitConflict = checkCommitConflict; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java index 6816868c6e9d..6ee92fee6c4d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java @@ -37,6 +37,6 @@ public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, this.checkCommitConflict = checkCommitConflict; } - public void fresh(String batchInterval, String period, String markerDir, String basePath, - String maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) {} + public void fresh(Long batchInterval, Long period, String markerDir, String basePath, + Long maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) {} } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java index 10f2990d1e7b..81836bdb8523 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/marker/MarkerOperation.java @@ -39,12 +39,4 @@ public class MarkerOperation implements Serializable { // POST requests public static final String CREATE_MARKER_URL = String.format("%s/%s", BASE_URL, "create"); public static final String DELETE_MARKER_DIR_URL = String.format("%s/%s", BASE_URL, "dir/delete"); - - // marker conflict check related configs. - public static final String MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = "markerbatchinterval"; - public static final String MARKER_CONFLICT_CHECKER_PERIOD = "markercheckperiod"; - public static final String MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL = "markerheartbeatinterval"; - public static final String MARKER_CONFLICT_CHECKER_ENABLE = "markerconflictenable"; - public static final String MARKER_CONFLICT_CHECKER_STRATEGY = "markerearlyconflictdetectionstrategy"; - public static final String MARKER_CONFLICT_CHECK_COMMIT_CONFLICT = "markercheckcommitconflict"; } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index d146df1bc474..edab93342bd2 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -461,13 +461,7 @@ private void registerMarkerAPI() { ctx, ctx.queryParamAsClass(MarkerOperation.MARKER_DIR_PATH_PARAM, String.class).getOrDefault(""), ctx.queryParamAsClass(MarkerOperation.MARKER_NAME_PARAM, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_PERIOD, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_HEART_BEAT_INTERVAL, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_BASEPATH_PARAM, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_ENABLE, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECKER_STRATEGY, String.class).getOrDefault(""), - ctx.queryParamAsClass(MarkerOperation.MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.class).getOrDefault(""))); + ctx.queryParamAsClass(MarkerOperation.MARKER_BASEPATH_PARAM, String.class).getOrDefault(""))); }, false)); app.post(MarkerOperation.DELETE_MARKER_DIR_URL, new ViewHandler(ctx -> { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 4b884ee25ef6..86a4d0e4cbf3 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -122,6 +122,27 @@ public static class Config implements Serializable { @Parameter(names = {"--marker-parallelism", "-mdp"}, description = "Parallelism to use for reading and deleting marker files") public int markerParallelism = 100; + @Parameter(names = {"--early-conflict-detection-strategy"}, description = "Early conflict detection class name, this should be subclass of " + + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy") + public String earlyConflictDetectStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; + + @Parameter(names = {"--early-conflict-detection-check-commit-conflict"}, description = "Enable check commit conflict or not during early conflict detect") + public Boolean checkCommitConflict = false; + + @Parameter(names = {"--early-conflict-detection-enable"}, description = "Enable early conflict detection based on markers. It will try to detect writing conflict before create markers and fast fail " + + "which will release cluster resources as soon as possible.") + public Boolean earlyConflictDetectionEnable = false; + + @Parameter(names = {"--early-conflict-async-checker-batch-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The time to delay first async marker conflict checking.") + public Long earlyConflictAsyncCheckerBatchInterval = 30000L; + + @Parameter(names = {"--early-conflict-async-checker-batch-period"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking.") + public Long earlyConflictAsyncCheckerBatchPeriod = 30000L; + + @Parameter(names = {"--early-conflict-async-checker-heart-beat-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. " + + "Instants whose heartbeat is greater than the current value will not be used in early conflict detection.") + public Long maxAllowableHeartbeatIntervalInMs = 60000L; + @Parameter(names = {"--help", "-h"}) public Boolean help = false; @@ -146,6 +167,12 @@ public static class Builder { private int markerBatchNumThreads = 20; private long markerBatchIntervalMs = 50L; private int markerParallelism = 100; + private String earlyConflictDetectStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; + private Boolean checkCommitConflict = false; + private Boolean earlyConflictDetectionEnable = false; + private Long earlyConflictAsyncCheckerBatchInterval = 30000L; + private Long earlyConflictAsyncCheckerBatchPeriod = 30000L; + private Long maxAllowableHeartbeatIntervalInMs = 60000L; public Builder() { } @@ -215,6 +242,36 @@ public Builder markerParallelism(int markerParallelism) { return this; } + public Builder markerEarlyConflictDetectStrategy(String earlyConflictDetectStrategy) { + this.earlyConflictDetectStrategy = earlyConflictDetectStrategy; + return this; + } + + public Builder markerEarlyConflictDetectCheckCommitConflict(Boolean checkCommitConflict) { + this.checkCommitConflict = checkCommitConflict; + return this; + } + + public Builder markerEarlyConflictDetectEnable(Boolean earlyConflictDetectionEnable) { + this.earlyConflictDetectionEnable = earlyConflictDetectionEnable; + return this; + } + + public Builder markerEarlyConflictAsyncCheckerBatchInterval(Long earlyConflictAsyncCheckerBatchInterval) { + this.earlyConflictAsyncCheckerBatchInterval = earlyConflictAsyncCheckerBatchInterval; + return this; + } + + public Builder markerEarlyConflictAsyncCheckerBatchPeriod(Long earlyConflictAsyncCheckerBatchPeriod) { + this.earlyConflictAsyncCheckerBatchPeriod = earlyConflictAsyncCheckerBatchPeriod; + return this; + } + + public Builder markerEarlyConflictMaxAllowableHeartbeatIntervalInMs(Long maxAllowableHeartbeatIntervalInMs) { + this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; + return this; + } + public Config build() { Config config = new Config(); config.serverPort = this.serverPort; @@ -230,6 +287,12 @@ public Config build() { config.markerBatchNumThreads = this.markerBatchNumThreads; config.markerBatchIntervalMs = this.markerBatchIntervalMs; config.markerParallelism = this.markerParallelism; + config.earlyConflictDetectStrategy = this.earlyConflictDetectStrategy; + config.checkCommitConflict = this.checkCommitConflict; + config.earlyConflictDetectionEnable = this.earlyConflictDetectionEnable; + config.earlyConflictAsyncCheckerBatchInterval = this.earlyConflictAsyncCheckerBatchInterval; + config.earlyConflictAsyncCheckerBatchPeriod = this.earlyConflictAsyncCheckerBatchPeriod; + config.maxAllowableHeartbeatIntervalInMs = this.maxAllowableHeartbeatIntervalInMs; return config; } } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 334b8f68b24a..926531896373 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -165,18 +165,14 @@ public boolean doesMarkerDirExist(String markerDir) { * @param markerName marker name * @return the {@code CompletableFuture} instance for the request */ - public CompletableFuture createMarker(Context context, String markerDir, String markerName, - String batchInterval, String period, String maxAllowableHeartbeatIntervalInMs, - String basePath, String earlyConflictDetectionEnable, - String earlyConflictDetectionClassName, - String checkCommitConflict) { + public CompletableFuture createMarker(Context context, String markerDir, String markerName, String basePath) { // Step1 do early conflict detection if enable - if (Boolean.parseBoolean(earlyConflictDetectionEnable)) { + if (timelineServiceConfig.earlyConflictDetectionEnable) { try { synchronized (earlyConflictDetectionLock) { if (earlyConflictDetectionStrategy == null) { - earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(earlyConflictDetectionClassName, - basePath, markerDir, markerName, Boolean.parseBoolean(checkCommitConflict)); + earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(timelineServiceConfig.earlyConflictDetectStrategy, + basePath, markerDir, markerName, timelineServiceConfig.checkCommitConflict); } // markerDir => $base_path/.hoodie/.temp/$instant_time @@ -192,7 +188,8 @@ public CompletableFuture createMarker(Context context, String markerDir, .getInstants() .collect(Collectors.toSet()); - earlyConflictDetectionStrategy.fresh(batchInterval, period, markerDir, basePath, maxAllowableHeartbeatIntervalInMs, fileSystem, + earlyConflictDetectionStrategy.fresh(timelineServiceConfig.earlyConflictAsyncCheckerBatchInterval, + timelineServiceConfig.earlyConflictAsyncCheckerBatchPeriod, markerDir, basePath, timelineServiceConfig.maxAllowableHeartbeatIntervalInMs, fileSystem, this, oldInstants); } } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index a71d49a41769..7b4050db0c89 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -54,16 +54,16 @@ public void resolveMarkerConflict(String basePath, String markerDir, String mark throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); } - public void fresh(String batchInterval, String period, String markerDir, String basePath, - String maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) { + public void fresh(Long batchInterval, Long period, String markerDir, String basePath, + Long maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) { if (markerChecker != null) { markerChecker.shutdown(); } hasConflict.set(false); markerChecker = Executors.newSingleThreadScheduledExecutor(); markerChecker.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, - fileSystem, Long.parseLong(maxAllowableHeartbeatIntervalInMs), oldInstants, checkCommitConflict), - Long.parseLong(batchInterval), Long.parseLong(period), TimeUnit.MILLISECONDS); + fileSystem, maxAllowableHeartbeatIntervalInMs, oldInstants, checkCommitConflict), + batchInterval, period, TimeUnit.MILLISECONDS); } @Override diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index 0a6889e56d88..810be06b4628 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; @@ -79,8 +80,9 @@ public void run() { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(new Configuration()).setBasePath(basePath) .setLoadActiveTimelineOnLoad(true).build(); + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - List candidate = MarkerUtils.getCandidateInstants(metaClient.getActiveTimeline(), instants, + List candidate = MarkerUtils.getCandidateInstants(activeTimeline, instants, MarkerUtils.markerDirToInstantTime(markerDir), maxAllowableHeartbeatIntervalInMs, fs, basePath); Set tableMarkers = candidate.stream().flatMap(instant -> { return MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(instant, fs, new HoodieLocalEngineContext(new Configuration()), 100) @@ -92,7 +94,7 @@ public void run() { currentFileIDs.retainAll(tableFilesIDs); if (!currentFileIDs.isEmpty() - || (checkCommitConflict && MarkerUtils.hasCommitConflict(metaClient.getActiveTimeline(), + || (checkCommitConflict && MarkerUtils.hasCommitConflict(activeTimeline, currentInstantAllMarkers.stream().map(MarkerUtils::makerToPartitionAndFileID).collect(Collectors.toSet()), oldInstants))) { LOG.warn("Conflict writing detected based on markers!\n" + "Conflict markers: " + currentInstantAllMarkers + "\n" From 869baf7be4f626d10ca20606f7816b941203ec3a Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 22 Nov 2022 22:41:32 +0800 Subject: [PATCH 34/51] address comments --- .../apache/hudi/timeline/service/TimelineService.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 86a4d0e4cbf3..71720485da9a 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -129,14 +129,16 @@ public static class Config implements Serializable { @Parameter(names = {"--early-conflict-detection-check-commit-conflict"}, description = "Enable check commit conflict or not during early conflict detect") public Boolean checkCommitConflict = false; - @Parameter(names = {"--early-conflict-detection-enable"}, description = "Enable early conflict detection based on markers. It will try to detect writing conflict before create markers and fast fail " - + "which will release cluster resources as soon as possible.") + @Parameter(names = {"--early-conflict-detection-enable"}, description = "Enable early conflict detection based on markers. It will try to detect writing conflict " + + "before create markers and fast fail which will release cluster resources as soon as possible.") public Boolean earlyConflictDetectionEnable = false; - @Parameter(names = {"--early-conflict-async-checker-batch-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The time to delay first async marker conflict checking.") + @Parameter(names = {"--early-conflict-async-checker-batch-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. " + + "The time to delay first async marker conflict checking.") public Long earlyConflictAsyncCheckerBatchInterval = 30000L; - @Parameter(names = {"--early-conflict-async-checker-batch-period"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking.") + @Parameter(names = {"--early-conflict-async-checker-batch-period"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy." + + " The period between each marker conflict checking.") public Long earlyConflictAsyncCheckerBatchPeriod = 30000L; @Parameter(names = {"--early-conflict-async-checker-heart-beat-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. " From 0447a7135e0cf136cd894291a1efd4374c3efd66 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 23 Nov 2022 09:44:23 +0800 Subject: [PATCH 35/51] address comments --- .../marker/TestMarkerBasedEarlyConflictDetectionRunnable.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java index 5da169ae0f70..9ba5ae2af1ef 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java @@ -21,8 +21,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.timeline.service.handlers.MarkerHandler; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -80,6 +82,7 @@ public void testMarkerCheckerRunnable() throws IOException, InterruptedException MarkerHandler markerHandler = mock(MarkerHandler.class); String rootBaseMarkerDir = basePath + "/.hoodie/.temp"; String partition = "2016"; + metaClient = HoodieTestUtils.init(new Configuration(), basePath, HoodieTableType.COPY_ON_WRITE); String oldInstant = "001"; Set oldMarkers = Stream.of(partition + "/b21adfa2-7013-4452-a565-4cc39fea5b73-0_4-17-21_001.parquet.marker.CREATE", @@ -108,6 +111,7 @@ public void testMarkerCheckerRunnable() throws IOException, InterruptedException } private void prepareFiles(String baseMarkerDir, String instant, Set markers, FileSystem fs) throws IOException { + fs.create(new Path(basePath + "/.hoodie/" + instant + ".commit"), true); String markerDir = baseMarkerDir + "/" + instant; fs.mkdirs(new Path(markerDir)); BufferedWriter out = new BufferedWriter(new FileWriter(markerDir + "/MARKERS0")); From c973c81cc138a320e562f293f1dc72828ed8fac6 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 5 Jan 2023 22:52:58 -0800 Subject: [PATCH 36/51] Fix errors after rebase --- .../hudi/table/marker/WriteMarkers.java | 9 +++--- .../client/TestHoodieClientMultiWriter.java | 13 +++++---- .../apache/hudi/common/util/MarkerUtils.java | 3 +- .../service/handlers/MarkerHandler.java | 29 ++++++++++--------- 4 files changed, 29 insertions(+), 25 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 99d17d3cfdc8..a6fefd458fd0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -26,17 +26,17 @@ 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.hadoop.fs.Path; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; + +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.io.Serializable; +import java.util.HashSet; import java.util.Set; -import java.util.stream.Collectors; /** * Operates on markers for a given write action (commit, delta commit, compaction). @@ -89,7 +89,8 @@ public Option create(String partitionPath, String dataFileName, IOType typ return create(partitionPath, dataFileName, type, checkIfExists); } - Set completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + Set completedCommitInstants = new HashSet<>( + activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants()); String fileId = handler.get().getFileId(); HoodieWriteConfig config = handler.get().getConfig(); 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 index eab071d9abcd..f82cb18e782a 100644 --- 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 @@ -18,10 +18,10 @@ package org.apache.hudi.client; -import org.apache.curator.test.TestingServer; import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; @@ -37,21 +37,21 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.table.marker.SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; +import org.apache.curator.test.TestingServer; import org.apache.hadoop.fs.Path; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; import org.apache.spark.SparkException; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; @@ -237,7 +237,8 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta }); List completedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .filterCompletedInstants().getInstants().stream() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); assertEquals(3, completedInstant.size()); assertTrue(completedInstant.contains(nextCommitTime1)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 7678fc62443f..a524dfa241f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -236,7 +236,8 @@ public static List getAllMarkerDir(Path tempPath, FileSystem fs) throws IO public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set currentFileIDs, Set completedCommitInstants) { - Set currentInstants = activeTimeline.reload().getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + Set currentInstants = new HashSet<>( + activeTimeline.reload().getCommitsTimeline().filterCompletedInstants().getInstants()); currentInstants.removeAll(completedCommitInstants); Set missingFileIDs = currentInstants.stream().flatMap(instant -> { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 926531896373..7a5fce4c8945 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -18,13 +18,6 @@ package org.apache.hudi.timeline.service.handlers; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_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; -import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; @@ -39,6 +32,8 @@ import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationFuture; import org.apache.hudi.timeline.service.handlers.marker.MarkerDirState; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import io.javalin.http.Context; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -46,10 +41,11 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; +import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -57,6 +53,11 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_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; +import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult; + /** * REST Handler servicing marker requests. * @@ -181,12 +182,12 @@ public CompletableFuture createMarker(Context context, String markerDir, if (!markerDir.equalsIgnoreCase(currentMarkerDir)) { this.currentMarkerDir = markerDir; Set actions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION); - Set oldInstants = viewManager.getFileSystemView(basePath) - .getTimeline() - .filterCompletedInstants() - .filter(instant -> actions.contains(instant.getAction())) - .getInstants() - .collect(Collectors.toSet()); + Set oldInstants = new HashSet<>( + viewManager.getFileSystemView(basePath) + .getTimeline() + .filterCompletedInstants() + .filter(instant -> actions.contains(instant.getAction())) + .getInstants()); earlyConflictDetectionStrategy.fresh(timelineServiceConfig.earlyConflictAsyncCheckerBatchInterval, timelineServiceConfig.earlyConflictAsyncCheckerBatchPeriod, markerDir, basePath, timelineServiceConfig.maxAllowableHeartbeatIntervalInMs, fileSystem, From c412635478d07b2bb37ad6738c06882f84b54001 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sun, 8 Jan 2023 17:04:30 -0800 Subject: [PATCH 37/51] Improve abstraction for lock and transaction manager, rename configs, and revise config description --- .../embedded/EmbeddedTimelineService.java | 13 ++- .../hudi/client/heartbeat/HeartbeatUtils.java | 9 +- .../heartbeat/HoodieHeartbeatClient.java | 22 ++--- .../DirectMarkerTransactionManager.java | 82 +++++++++++++++++++ .../transaction/TransactionManager.java | 44 ++++------ .../client/transaction/lock/LockManager.java | 52 +++--------- .../apache/hudi/config/HoodieWriteConfig.java | 55 +++++++------ ...erBasedEarlyConflictDetectionStrategy.java | 10 ++- .../TimelineServerBasedWriteMarkers.java | 6 +- .../client/TestHoodieClientMultiWriter.java | 8 +- .../heartbeat/HoodieHeartbeatUtils.java | 65 +++++++++++++++ .../apache/hudi/common/util/MarkerUtils.java | 29 +------ .../timeline/service/TimelineService.java | 66 ++++++++------- .../service/handlers/MarkerHandler.java | 6 +- 14 files changed, 280 insertions(+), 187 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index 6b70afcc3c1a..c913bdd97c5e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -86,16 +86,15 @@ public void startServer() throws IOException { .enableMarkerRequests(true) .markerBatchNumThreads(writeConfig.getMarkersTimelineServerBasedBatchNumThreads()) .markerBatchIntervalMs(writeConfig.getMarkersTimelineServerBasedBatchIntervalMs()) - .markerParallelism(writeConfig.getMarkersDeleteParallelism()) - ; + .markerParallelism(writeConfig.getMarkersDeleteParallelism()); } if (writeConfig.isEarlyConflictDetectionEnable()) { - timelineServiceConfBuilder.markerEarlyConflictDetectEnable(true) - .markerEarlyConflictDetectStrategy(writeConfig.getEarlyConflictDetectionStrategyClassName()) - .markerEarlyConflictDetectCheckCommitConflict(writeConfig.earlyConflictDetectionCheckCommitConflict()) - .markerEarlyConflictAsyncCheckerBatchInterval(writeConfig.getEarlyConflictDetectionAsyncCheckerBatchInterval()) - .markerEarlyConflictAsyncCheckerBatchPeriod(writeConfig.getEarlyConflictDetectionAsyncCheckerPeriod()) + timelineServiceConfBuilder.earlyConflictDetectionEnable(true) + .earlyConflictDetectionStrategy(writeConfig.getEarlyConflictDetectionStrategyClassName()) + .earlyConflictDetectionCheckCommitConflict(writeConfig.earlyConflictDetectionCheckCommitConflict()) + .asyncConflictDetectorBatchIntervalMs(writeConfig.getAsyncConflictDetectorBatchIntervalMs()) + .asyncConflictDetectorBatchPeriodMs(writeConfig.getAsyncConflictDetectorPeriodMs()) .markerEarlyConflictMaxAllowableHeartbeatIntervalInMs(writeConfig.getHoodieClientHeartbeatIntervalInMs()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java index a20469429030..eb12ab634fda 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java @@ -18,18 +18,21 @@ package org.apache.hudi.client.heartbeat; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; +import static org.apache.hudi.common.heartbeat.HoodieHeartbeatUtils.getLastHeartbeatTime; + /** * Helper class to delete heartbeat for completed or failed instants with expired heartbeats. */ @@ -89,7 +92,7 @@ public static void abortIfHeartbeatExpired(String instantTime, HoodieTable table try { if (config.getFailedWritesCleanPolicy().isLazy() && heartbeatClient.isHeartbeatExpired(instantTime)) { throw new HoodieException("Heartbeat for instant " + instantTime + " has expired, last heartbeat " - + HoodieHeartbeatClient.getLastHeartbeatTime(table.getMetaClient().getFs(), config.getBasePath(), instantTime)); + + getLastHeartbeatTime(table.getMetaClient().getFs(), config.getBasePath(), instantTime)); } } catch (IOException io) { throw new HoodieException("Unable to read heartbeat", io); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java index 341d72c754a9..50d83bb13980 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java @@ -18,17 +18,19 @@ package org.apache.hudi.client.heartbeat; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieHeartbeatException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import javax.annotation.concurrent.NotThreadSafe; + import java.io.IOException; import java.io.OutputStream; import java.io.Serializable; @@ -37,9 +39,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import java.util.Timer; import java.util.TimerTask; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.heartbeat.HoodieHeartbeatUtils.getLastHeartbeatTime; /** * This class creates heartbeat for hudi client. This heartbeat is used to ascertain whether the running job is or not. @@ -205,16 +209,6 @@ public void stop() throws HoodieException { instantToHeartbeatMap.values().forEach(heartbeat -> stop(heartbeat.getInstantTime())); } - public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { - Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); - if (fs.exists(heartbeatFilePath)) { - return fs.getFileStatus(heartbeatFilePath).getModificationTime(); - } else { - // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed - return 0L; - } - } - public static Boolean heartbeatExists(FileSystem fs, String basePath, String instantTime) throws IOException { Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); if (fs.exists(heartbeatFilePath)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java new file mode 100644 index 000000000000..0f5612f414e8 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hudi.client.transaction.lock.LockManager; +import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; + +import org.apache.hadoop.fs.FileSystem; + +import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; + +public class DirectMarkerTransactionManager extends TransactionManager { + private final String filePath; + + public DirectMarkerTransactionManager(HoodieWriteConfig config, FileSystem fs, String partitionPath, String fileId) { + super(new LockManager(config, fs, createUpdatedLockProps(config, partitionPath, fileId)), + config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()); + this.filePath = partitionPath + "/" + fileId; + } + + public void beginTransaction(String newTxnOwnerInstantTime) { + if (isOptimisticConcurrencyControlEnabled) { + LOG.info("Transaction starting for " + newTxnOwnerInstantTime + " and " + filePath); + lockManager.lock(); + + reset(currentTxnOwnerInstant, Option.of(getInstant(newTxnOwnerInstantTime)), Option.empty()); + LOG.info("Transaction started for " + newTxnOwnerInstantTime + " and " + filePath); + } + } + + public void endTransaction(String currentTxnOwnerInstantTime) { + if (isOptimisticConcurrencyControlEnabled) { + LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstantTime + + " for " + filePath); + if (reset(Option.of(getInstant(currentTxnOwnerInstantTime)), Option.empty(), Option.empty())) { + lockManager.unlock(); + LOG.info("Transaction ended with transaction owner " + currentTxnOwnerInstantTime + + " for " + filePath); + } + } + } + + /** + * Rebuild lock related configs, only support ZK related lock for now. + */ + private static TypedProperties createUpdatedLockProps( + HoodieWriteConfig writeConfig, String partitionPath, String fileId) { + if (ZookeeperBasedLockProvider.class.getName().equals(writeConfig.getLockProviderClass())) { + throw new HoodieNotSupportedException("Only Support ZK-based lock for DirectMarkerTransactionManager now."); + } + TypedProperties props = new TypedProperties(writeConfig.getProps()); + props.setProperty(LockConfiguration.ZK_LOCK_KEY_PROP_KEY, partitionPath + "/" + fileId); + return props; + } + + private HoodieInstant getInstant(String instantTime) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, EMPTY_STRING, instantTime); + } +} 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 index e778f0c4adfa..7fddf8a944b0 100644 --- 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 @@ -18,11 +18,12 @@ 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.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -34,20 +35,20 @@ */ public class TransactionManager implements Serializable { - private static final Logger LOG = LogManager.getLogger(TransactionManager.class); - private final LockManager lockManager; - private final boolean isOptimisticConcurrencyControlEnabled; - private Option currentTxnOwnerInstant = Option.empty(); + protected static final Logger LOG = LogManager.getLogger(TransactionManager.class); + protected final LockManager lockManager; + protected final boolean isOptimisticConcurrencyControlEnabled; + protected Option currentTxnOwnerInstant = Option.empty(); private Option lastCompletedTxnOwnerInstant = Option.empty(); public TransactionManager(HoodieWriteConfig config, FileSystem fs) { - this.lockManager = new LockManager(config, fs); - this.isOptimisticConcurrencyControlEnabled = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + this(new LockManager(config, fs), + config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()); } - public TransactionManager(HoodieWriteConfig config, FileSystem fs, String partitionPath, String fileId) { - this.lockManager = new LockManager(config, fs, partitionPath, fileId); - this.isOptimisticConcurrencyControlEnabled = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + protected TransactionManager(LockManager lockManager, boolean isOptimisticConcurrencyControlEnabled) { + this.lockManager = lockManager; + this.isOptimisticConcurrencyControlEnabled = isOptimisticConcurrencyControlEnabled; } public void beginTransaction(Option newTxnOwnerInstant, @@ -62,14 +63,6 @@ public void beginTransaction(Option newTxnOwnerInstant, } } - public void beginTransaction(String partitionPath, String fileId) { - if (isOptimisticConcurrencyControlEnabled) { - LOG.info("Transaction starting for " + partitionPath + "/" + fileId); - lockManager.lock(); - LOG.info("Transaction started for " + partitionPath + "/" + fileId); - } - } - public void endTransaction(Option currentTxnOwnerInstant) { if (isOptimisticConcurrencyControlEnabled) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant); @@ -80,18 +73,9 @@ public void endTransaction(Option currentTxnOwnerInstant) { } } - public void endTransaction(String partitionPath, String fileId) { - if (isOptimisticConcurrencyControlEnabled) { - String filePath = partitionPath + "/" + fileId; - LOG.info("Transaction ending with transaction for " + filePath); - lockManager.unlock(); - LOG.info("Transaction ended with transaction for " + filePath); - } - } - - private synchronized boolean reset(Option callerInstant, - Option newTxnOwnerInstant, - Option lastCompletedTxnOwnerInstant) { + protected synchronized boolean reset(Option callerInstant, + Option newTxnOwnerInstant, + Option lastCompletedTxnOwnerInstant) { if (!this.currentTxnOwnerInstant.isPresent() || this.currentTxnOwnerInstant.get().equals(callerInstant.get())) { this.currentTxnOwnerInstant = newTxnOwnerInstant; this.lastCompletedTxnOwnerInstant = lastCompletedTxnOwnerInstant; 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 index 378397ca008e..effcdf091ae2 100644 --- 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 @@ -18,20 +18,17 @@ package org.apache.hudi.client.transaction.lock; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.transaction.lock.metrics.HoodieLockMetrics; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieLockException; import org.apache.hadoop.fs.FileSystem; -import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -47,52 +44,27 @@ public class LockManager implements Serializable, AutoCloseable { private static final Logger LOG = LogManager.getLogger(LockManager.class); - private HoodieWriteConfig writeConfig; - private LockConfiguration lockConfiguration; - private SerializableConfiguration hadoopConf; - private int maxRetries; - private long maxWaitTimeInMs; + private final HoodieWriteConfig writeConfig; + private final LockConfiguration lockConfiguration; + private final SerializableConfiguration hadoopConf; + private final int maxRetries; + private final long maxWaitTimeInMs; private transient HoodieLockMetrics metrics; private volatile LockProvider lockProvider; public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { - init(writeConfig, fs.getConf(), writeConfig.getProps()); + this(writeConfig, fs, writeConfig.getProps()); } - /** - * Try to have a lock at partitionPath + fileID level for different write handler. - * @param writeConfig - * @param fs - * @param partitionPath - * @param fileId - */ - public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, String partitionPath, String fileId) { - TypedProperties props = refreshLockConfig(writeConfig, partitionPath + "/" + fileId); - init(writeConfig, fs.getConf(), props); - } - - private void init(HoodieWriteConfig writeConfig, Configuration conf, TypedProperties lockProps) { - this.lockConfiguration = new LockConfiguration(lockProps); + public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, TypedProperties lockProps) { this.writeConfig = writeConfig; - this.hadoopConf = new SerializableConfiguration(conf); - this.maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, + this.hadoopConf = new SerializableConfiguration(fs.getConf()); + this.lockConfiguration = new LockConfiguration(lockProps); + maxRetries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, Integer.parseInt(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_NUM_RETRIES.defaultValue())); - this.maxWaitTimeInMs = lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, + maxWaitTimeInMs = lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, Long.parseLong(HoodieLockConfig.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS.defaultValue())); - this.metrics = new HoodieLockMetrics(writeConfig); - } - - /** - * rebuild lock related configs, only support ZK related lock for now. - */ - private TypedProperties refreshLockConfig(HoodieWriteConfig writeConfig, String key) { - TypedProperties props = new TypedProperties(writeConfig.getProps()); - String zkBasePath = props.getProperty(LockConfiguration.ZK_BASE_PATH_PROP_KEY); - if (StringUtils.isNullOrEmpty(zkBasePath)) { - throw new HoodieNotSupportedException("Only Support ZK based lock for now."); - } - props.setProperty(LockConfiguration.ZK_LOCK_KEY_PROP_KEY, key); - return props; + metrics = new HoodieLockMetrics(writeConfig); } public void lock() { 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 b3c099da75ba..18ed7ce86c61 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 @@ -75,9 +75,9 @@ import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.table.storage.HoodieStorageLayout; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; import org.apache.hadoop.hbase.io.compress.Compression; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.orc.CompressionKind; @@ -568,33 +568,40 @@ public class HoodieWriteConfig extends HoodieConfig { return Option.of(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()); } }) - .withDocumentation("Early conflict detection class name, this should be subclass of " - + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy"); + .withDocumentation("The class name of the early conflict detection strategy to use. " + + "This should be a subclass of " + + "`org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy`."); public static final ConfigProperty EARLY_CONFLICT_DETECTION_ENABLE = ConfigProperty .key(CONCURRENCY_PREFIX + "early.conflict.detection.enable") .defaultValue(false) .sinceVersion("0.13.0") - .withDocumentation("Enable early conflict detection based on markers. It will try to detect writing conflict before create markers and fast fail" - + " which will release cluster resources as soon as possible."); + .withDocumentation("Whether to enable early conflict detection based on markers. " + + "It eagerly detects writing conflict before create markers and fails fast if a " + + "conflict is detected, to release cluster compute resources as soon as possible."); - public static final ConfigProperty MARKER_CONFLICT_CHECKER_BATCH_INTERVAL = ConfigProperty - .key(CONCURRENCY_PREFIX + "early.conflict.async.checker.batch.interval") + public static final ConfigProperty ASYNC_CONFLICT_DETECTOR_BATCH_INTERVAL_MS = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.async.detector.batch.interval_ms") .defaultValue(30000L) .sinceVersion("0.13.0") - .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The time to delay first async marker conflict checking."); + .withDocumentation("Used for timeline-server-based markers with " + + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "The time in milliseconds to delay first async marker conflict detection."); - public static final ConfigProperty MARKER_CONFLICT_CHECKER_PERIOD = ConfigProperty - .key(CONCURRENCY_PREFIX + "early.conflict.async.checker.period") + public static final ConfigProperty ASYNC_CONFLICT_DETECTOR_PERIOD_MS = ConfigProperty + .key(CONCURRENCY_PREFIX + "early.conflict.async.checker.period_ms") .defaultValue(30000L) .sinceVersion("0.13.0") - .withDocumentation("Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. The period between each marker conflict checking."); + .withDocumentation("Used for timeline-server-based markers with " + + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "The period in milliseconds between consecutive runs of async marker conflict detection."); - public static final ConfigProperty MARKER_CONFLICT_CHECK_COMMIT_CONFLICT = ConfigProperty + public static final ConfigProperty EARLY_CONFLICT_DETECTION_CHECK_COMMIT_CONFLICT = ConfigProperty .key(CONCURRENCY_PREFIX + "early.conflict.check.commit.conflict") .defaultValue(false) .sinceVersion("0.13.0") - .withDocumentation("Enable check commit conflict or not during early conflict detect"); + .withDocumentation("Whether to enable commit conflict checking or not during early " + + "conflict detection."); private ConsistencyGuardConfig consistencyGuardConfig; @@ -2236,12 +2243,12 @@ public ConflictResolutionStrategy getWriteConflictResolutionStrategy() { return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME)); } - public Long getEarlyConflictDetectionAsyncCheckerBatchInterval() { - return getLong(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL); + public Long getAsyncConflictDetectorBatchIntervalMs() { + return getLong(ASYNC_CONFLICT_DETECTOR_BATCH_INTERVAL_MS); } - public Long getEarlyConflictDetectionAsyncCheckerPeriod() { - return getLong(MARKER_CONFLICT_CHECKER_PERIOD); + public Long getAsyncConflictDetectorPeriodMs() { + return getLong(ASYNC_CONFLICT_DETECTOR_PERIOD_MS); } public Long getLockAcquireWaitTimeoutInMs() { @@ -2261,7 +2268,7 @@ public String getEarlyConflictDetectionStrategyClassName() { } public boolean earlyConflictDetectionCheckCommitConflict() { - return getBoolean(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT); + return getBoolean(EARLY_CONFLICT_DETECTION_CHECK_COMMIT_CONFLICT); } // misc configs @@ -2796,18 +2803,18 @@ public Builder withEarlyConflictDetectionEnable(boolean enable) { return this; } - public Builder withMarkerConflictCheckerBatchInterval(long interval) { - writeConfig.setValue(MARKER_CONFLICT_CHECKER_BATCH_INTERVAL, String.valueOf(interval)); + public Builder withAsyncConflictDetectorBatchIntervalMs(long intervalMs) { + writeConfig.setValue(ASYNC_CONFLICT_DETECTOR_BATCH_INTERVAL_MS, String.valueOf(intervalMs)); return this; } - public Builder withMarkerConflictCheckerPeriod(long period) { - writeConfig.setValue(MARKER_CONFLICT_CHECKER_PERIOD, String.valueOf(period)); + public Builder withAsyncConflictDetectorPeriodMs(long periodMs) { + writeConfig.setValue(ASYNC_CONFLICT_DETECTOR_PERIOD_MS, String.valueOf(periodMs)); return this; } - public Builder withCheckCommitConflict(boolean enable) { - writeConfig.setValue(MARKER_CONFLICT_CHECK_COMMIT_CONFLICT, String.valueOf(enable)); + public Builder withEarlyConflictDetectionCheckCommitConflict(boolean enable) { + writeConfig.setValue(EARLY_CONFLICT_DETECTION_CHECK_COMMIT_CONFLICT, String.valueOf(enable)); return this; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index bfd60fe3ddac..b6ec783635ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -18,11 +18,12 @@ package org.apache.hudi.table.marker; -import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.client.transaction.DirectMarkerTransactionManager; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieWriteConfig; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -44,10 +45,11 @@ public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String b @Override public void detectAndResolveConflictIfNecessary() { - TransactionManager txnManager = new TransactionManager((HoodieWriteConfig) config, fs, partitionPath, fileId); + DirectMarkerTransactionManager txnManager = + new DirectMarkerTransactionManager((HoodieWriteConfig) config, fs, partitionPath, fileId); try { // Need to do transaction before create marker file when using early conflict detection - txnManager.beginTransaction(partitionPath, fileId); + txnManager.beginTransaction(instantTime); super.detectAndResolveConflictIfNecessary(); } catch (Exception e) { @@ -55,7 +57,7 @@ public void detectAndResolveConflictIfNecessary() { throw e; } finally { // End transaction after created marker file. - txnManager.endTransaction(partitionPath, fileId); + txnManager.endTransaction(instantTime); txnManager.close(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 7fc76d732a0e..6743e56d59d2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -190,11 +190,13 @@ private boolean executeCreateMarkerRequest(Map paramsMap, String /** * init create marker related config maps. + * * @param partitionPath * @param markerFileName * @return */ - private Map initConfigMap(String partitionPath, String markerFileName, boolean initEarlyConflictConfigs) { + private Map initConfigMap( + String partitionPath, String markerFileName, boolean initEarlyConflictDetectionConfigs) { Map paramsMap = new HashMap<>(); paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); @@ -204,7 +206,7 @@ private Map initConfigMap(String partitionPath, String markerFil paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName); } - if (initEarlyConflictConfigs) { + if (initEarlyConflictDetectionConfigs) { paramsMap.put(MARKER_BASEPATH_PARAM, basePath); } 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 index f82cb18e782a..3bf0669e75b6 100644 --- 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 @@ -832,8 +832,8 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withMarkersType(MarkerType.DIRECT.name()) .withEarlyConflictDetectionEnable(true) .withEarlyConflictDetectionStrategy(earlyConflictDetectionStrategy) - .withMarkerConflictCheckerBatchInterval(0) - .withMarkerConflictCheckerPeriod(100) + .withAsyncConflictDetectorBatchIntervalMs(0) + .withAsyncConflictDetectorPeriodMs(100) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(lockProvider).build()) .withAutoCommit(false).withProperties(properties).build(); } else { @@ -853,8 +853,8 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(lockProvider).build()) .withEarlyConflictDetectionEnable(true) .withEarlyConflictDetectionStrategy(earlyConflictDetectionStrategy) - .withMarkerConflictCheckerBatchInterval(0) - .withMarkerConflictCheckerPeriod(100) + .withAsyncConflictDetectorBatchIntervalMs(0) + .withAsyncConflictDetectorPeriodMs(100) .withAutoCommit(false).withProperties(properties).build(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java new file mode 100644 index 000000000000..49c1483756ed --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java @@ -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.common.heartbeat; + +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; + +/** + * Common utils for Hudi heartbeat + */ +public class HoodieHeartbeatUtils { + private static final Logger LOG = LogManager.getLogger(HoodieHeartbeatUtils.class); + + /** + * Use modification time as last heart beat time + * + * @param fs + * @param basePath + * @param instantTime + * @return + * @throws IOException + */ + public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); + if (fs.exists(heartbeatFilePath)) { + return fs.getFileStatus(heartbeatFilePath).getModificationTime(); + } else { + // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed + return 0L; + } + } + + public static boolean isHeartbeatExpired(String instantTime, long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) throws IOException { + Long currentTime = System.currentTimeMillis(); + Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); + if (currentTime - lastHeartbeatTime > maxAllowableHeartbeatIntervalInMs) { + LOG.warn("Heartbeat expired, for instant: " + instantTime); + return true; + } + return false; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index a524dfa241f6..9b24a93e2de2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -52,6 +52,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.heartbeat.HoodieHeartbeatUtils.isHeartbeatExpired; import static org.apache.hudi.common.util.FileIOUtils.closeQuietly; /** @@ -304,32 +305,4 @@ public static String markerDirToInstantTime(String marker) { String[] ele = marker.split("/"); return ele[ele.length - 1]; } - - /** - * Use modification time as last heart beat time - * @param fs - * @param basePath - * @param instantTime - * @return - * @throws IOException - */ - public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { - Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime); - if (fs.exists(heartbeatFilePath)) { - return fs.getFileStatus(heartbeatFilePath).getModificationTime(); - } else { - // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed - return 0L; - } - } - - public static boolean isHeartbeatExpired(String instantTime, long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) throws IOException { - Long currentTime = System.currentTimeMillis(); - Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); - if (currentTime - lastHeartbeatTime > maxAllowableHeartbeatIntervalInMs) { - LOG.warn("Heartbeat expired, for instant: " + instantTime); - return true; - } - return false; - } } \ No newline at end of file diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 3ac0bf9e34a5..63d308dcfd1d 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -18,8 +18,6 @@ package org.apache.hudi.timeline.service; -import io.javalin.core.JavalinConfig; -import io.javalin.jetty.JettyServer; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; @@ -33,6 +31,8 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import io.javalin.Javalin; +import io.javalin.core.JavalinConfig; +import io.javalin.jetty.JettyServer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; @@ -124,24 +124,34 @@ public static class Config implements Serializable { @Parameter(names = {"--marker-parallelism", "-mdp"}, description = "Parallelism to use for reading and deleting marker files") public int markerParallelism = 100; - @Parameter(names = {"--early-conflict-detection-strategy"}, description = "Early conflict detection class name, this should be subclass of " - + "org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy") - public String earlyConflictDetectStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; + @Parameter(names = {"--early-conflict-detection-strategy"}, description = + "The class name of the early conflict detection strategy to use. " + + "This should be subclass of " + + "`org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy`") + public String earlyConflictDetectionStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; - @Parameter(names = {"--early-conflict-detection-check-commit-conflict"}, description = "Enable check commit conflict or not during early conflict detect") + @Parameter(names = {"--early-conflict-detection-check-commit-conflict"}, description = + "Whether to enable commit conflict checking or not during early " + + "conflict detection.") public Boolean checkCommitConflict = false; - @Parameter(names = {"--early-conflict-detection-enable"}, description = "Enable early conflict detection based on markers. It will try to detect writing conflict " - + "before create markers and fast fail which will release cluster resources as soon as possible.") + @Parameter(names = {"--early-conflict-detection-enable"}, description = + "Whether to enable early conflict detection based on markers. " + + "It eagerly detects writing conflict before create markers and fails fast if a " + + "conflict is detected, to release cluster compute resources as soon as possible.") public Boolean earlyConflictDetectionEnable = false; - @Parameter(names = {"--early-conflict-async-checker-batch-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. " - + "The time to delay first async marker conflict checking.") - public Long earlyConflictAsyncCheckerBatchInterval = 30000L; + @Parameter(names = {"--async-conflict-detector-batch-interval-ms"}, description = + "Used for timeline-server-based markers with " + + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "The time in milliseconds to delay first async marker conflict detection.") + public Long asyncConflictDetectorBatchIntervalMs = 30000L; - @Parameter(names = {"--early-conflict-async-checker-batch-period"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy." - + " The period between each marker conflict checking.") - public Long earlyConflictAsyncCheckerBatchPeriod = 30000L; + @Parameter(names = {"--async-conflict-detector-batch-period-ms"}, description = + "Used for timeline-server-based markers with " + + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "The period in milliseconds between consecutive runs of async marker conflict detection.") + public Long asyncConflictDetectorBatchPeriodMs = 30000L; @Parameter(names = {"--early-conflict-async-checker-heart-beat-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. " + "Instants whose heartbeat is greater than the current value will not be used in early conflict detection.") @@ -171,11 +181,11 @@ public static class Builder { private int markerBatchNumThreads = 20; private long markerBatchIntervalMs = 50L; private int markerParallelism = 100; - private String earlyConflictDetectStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; + private String earlyConflictDetectionStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; private Boolean checkCommitConflict = false; private Boolean earlyConflictDetectionEnable = false; - private Long earlyConflictAsyncCheckerBatchInterval = 30000L; - private Long earlyConflictAsyncCheckerBatchPeriod = 30000L; + private Long asyncConflictDetectorBatchIntervalMs = 30000L; + private Long asyncConflictDetectorBatchPeriodMs = 30000L; private Long maxAllowableHeartbeatIntervalInMs = 60000L; public Builder() { @@ -246,28 +256,28 @@ public Builder markerParallelism(int markerParallelism) { return this; } - public Builder markerEarlyConflictDetectStrategy(String earlyConflictDetectStrategy) { - this.earlyConflictDetectStrategy = earlyConflictDetectStrategy; + public Builder earlyConflictDetectionStrategy(String earlyConflictDetectionStrategy) { + this.earlyConflictDetectionStrategy = earlyConflictDetectionStrategy; return this; } - public Builder markerEarlyConflictDetectCheckCommitConflict(Boolean checkCommitConflict) { + public Builder earlyConflictDetectionCheckCommitConflict(Boolean checkCommitConflict) { this.checkCommitConflict = checkCommitConflict; return this; } - public Builder markerEarlyConflictDetectEnable(Boolean earlyConflictDetectionEnable) { + public Builder earlyConflictDetectionEnable(Boolean earlyConflictDetectionEnable) { this.earlyConflictDetectionEnable = earlyConflictDetectionEnable; return this; } - public Builder markerEarlyConflictAsyncCheckerBatchInterval(Long earlyConflictAsyncCheckerBatchInterval) { - this.earlyConflictAsyncCheckerBatchInterval = earlyConflictAsyncCheckerBatchInterval; + public Builder asyncConflictDetectorBatchIntervalMs(Long asyncConflictDetectorBatchIntervalMs) { + this.asyncConflictDetectorBatchIntervalMs = asyncConflictDetectorBatchIntervalMs; return this; } - public Builder markerEarlyConflictAsyncCheckerBatchPeriod(Long earlyConflictAsyncCheckerBatchPeriod) { - this.earlyConflictAsyncCheckerBatchPeriod = earlyConflictAsyncCheckerBatchPeriod; + public Builder asyncConflictDetectorBatchPeriodMs(Long asyncConflictDetectorBatchPeriodMs) { + this.asyncConflictDetectorBatchPeriodMs = asyncConflictDetectorBatchPeriodMs; return this; } @@ -291,11 +301,11 @@ public Config build() { config.markerBatchNumThreads = this.markerBatchNumThreads; config.markerBatchIntervalMs = this.markerBatchIntervalMs; config.markerParallelism = this.markerParallelism; - config.earlyConflictDetectStrategy = this.earlyConflictDetectStrategy; + config.earlyConflictDetectionStrategy = this.earlyConflictDetectionStrategy; config.checkCommitConflict = this.checkCommitConflict; config.earlyConflictDetectionEnable = this.earlyConflictDetectionEnable; - config.earlyConflictAsyncCheckerBatchInterval = this.earlyConflictAsyncCheckerBatchInterval; - config.earlyConflictAsyncCheckerBatchPeriod = this.earlyConflictAsyncCheckerBatchPeriod; + config.asyncConflictDetectorBatchIntervalMs = this.asyncConflictDetectorBatchIntervalMs; + config.asyncConflictDetectorBatchPeriodMs = this.asyncConflictDetectorBatchPeriodMs; config.maxAllowableHeartbeatIntervalInMs = this.maxAllowableHeartbeatIntervalInMs; return config; } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 7a5fce4c8945..2f950eea88b0 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -172,7 +172,7 @@ public CompletableFuture createMarker(Context context, String markerDir, try { synchronized (earlyConflictDetectionLock) { if (earlyConflictDetectionStrategy == null) { - earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(timelineServiceConfig.earlyConflictDetectStrategy, + earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(timelineServiceConfig.earlyConflictDetectionStrategy, basePath, markerDir, markerName, timelineServiceConfig.checkCommitConflict); } @@ -189,8 +189,8 @@ public CompletableFuture createMarker(Context context, String markerDir, .filter(instant -> actions.contains(instant.getAction())) .getInstants()); - earlyConflictDetectionStrategy.fresh(timelineServiceConfig.earlyConflictAsyncCheckerBatchInterval, - timelineServiceConfig.earlyConflictAsyncCheckerBatchPeriod, markerDir, basePath, timelineServiceConfig.maxAllowableHeartbeatIntervalInMs, fileSystem, + earlyConflictDetectionStrategy.fresh(timelineServiceConfig.asyncConflictDetectorBatchIntervalMs, + timelineServiceConfig.asyncConflictDetectorBatchPeriodMs, markerDir, basePath, timelineServiceConfig.maxAllowableHeartbeatIntervalInMs, fileSystem, this, oldInstants); } } From 6bb19747ad7841c458fd0f1adbbfb99d6be13a14 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sun, 8 Jan 2023 17:46:05 -0800 Subject: [PATCH 38/51] Replace checker naming --- .../embedded/EmbeddedTimelineService.java | 2 +- .../apache/hudi/config/HoodieWriteConfig.java | 2 +- .../timeline/service/TimelineService.java | 8 +++-- .../service/handlers/MarkerHandler.java | 3 +- ...eMarkerEarlyConflictDetectionStrategy.java | 15 +++++----- ...erBasedEarlyConflictDetectionRunnable.java | 9 +++--- ...erBasedEarlyConflictDetectionRunnable.java | 29 ++++++++++--------- 7 files changed, 37 insertions(+), 31 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index c913bdd97c5e..7051cb344ff9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -95,7 +95,7 @@ public void startServer() throws IOException { .earlyConflictDetectionCheckCommitConflict(writeConfig.earlyConflictDetectionCheckCommitConflict()) .asyncConflictDetectorBatchIntervalMs(writeConfig.getAsyncConflictDetectorBatchIntervalMs()) .asyncConflictDetectorBatchPeriodMs(writeConfig.getAsyncConflictDetectorPeriodMs()) - .markerEarlyConflictMaxAllowableHeartbeatIntervalInMs(writeConfig.getHoodieClientHeartbeatIntervalInMs()); + .earlyConflictDetectionMaxAllowableHeartbeatIntervalInMs(writeConfig.getHoodieClientHeartbeatIntervalInMs()); } server = new TimelineService(context, hadoopConf.newCopy(), timelineServiceConfBuilder.build(), 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 18ed7ce86c61..4a9426e9cde0 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 @@ -589,7 +589,7 @@ public class HoodieWriteConfig extends HoodieConfig { + "The time in milliseconds to delay first async marker conflict detection."); public static final ConfigProperty ASYNC_CONFLICT_DETECTOR_PERIOD_MS = ConfigProperty - .key(CONCURRENCY_PREFIX + "early.conflict.async.checker.period_ms") + .key(CONCURRENCY_PREFIX + "early.conflict.async.detector.period_ms") .defaultValue(30000L) .sinceVersion("0.13.0") .withDocumentation("Used for timeline-server-based markers with " diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 63d308dcfd1d..e3737d09d60a 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -153,8 +153,10 @@ public static class Config implements Serializable { + "The period in milliseconds between consecutive runs of async marker conflict detection.") public Long asyncConflictDetectorBatchPeriodMs = 30000L; - @Parameter(names = {"--early-conflict-async-checker-heart-beat-interval"}, description = "Used for timeline based marker AsyncTimelineMarkerConflictResolutionStrategy. " - + "Instants whose heartbeat is greater than the current value will not be used in early conflict detection.") + @Parameter(names = {"--early-conflict-detection-max-heartbeat-interval-ms"}, description = + "Used for timeline-server-based markers with " + + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "Instants whose heartbeat is greater than the current value will not be used in early conflict detection.") public Long maxAllowableHeartbeatIntervalInMs = 60000L; @Parameter(names = {"--help", "-h"}) @@ -281,7 +283,7 @@ public Builder asyncConflictDetectorBatchPeriodMs(Long asyncConflictDetectorBatc return this; } - public Builder markerEarlyConflictMaxAllowableHeartbeatIntervalInMs(Long maxAllowableHeartbeatIntervalInMs) { + public Builder earlyConflictDetectionMaxAllowableHeartbeatIntervalInMs(Long maxAllowableHeartbeatIntervalInMs) { this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; return this; } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 2f950eea88b0..67e4a611f339 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -178,7 +178,8 @@ public CompletableFuture createMarker(Context context, String markerDir, // markerDir => $base_path/.hoodie/.temp/$instant_time // If markerDir is changed like move to the next instant action, we need to fresh this earlyConflictDetectionStrategy. - // For specific instant related create marker action, we only call this check/fresh once instead of starting the checker for every request + // For specific instant related create marker action, we only call this check/fresh once + // instead of starting the conflict detector for every request if (!markerDir.equalsIgnoreCase(currentMarkerDir)) { this.currentMarkerDir = markerDir; Set actions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java index 7b4050db0c89..0bf059996060 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java @@ -18,11 +18,12 @@ package org.apache.hudi.timeline.service.handlers.marker; -import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.timeline.service.handlers.MarkerHandler; + +import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -38,7 +39,7 @@ public class AsyncTimelineMarkerEarlyConflictDetectionStrategy extends HoodieTim private static final Logger LOG = LogManager.getLogger(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class); private AtomicBoolean hasConflict = new AtomicBoolean(false); - private ScheduledExecutorService markerChecker; + private ScheduledExecutorService asyncDetectorExecutor; public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { super(basePath, markerDir, markerName, checkCommitConflict); @@ -56,13 +57,13 @@ public void resolveMarkerConflict(String basePath, String markerDir, String mark public void fresh(Long batchInterval, Long period, String markerDir, String basePath, Long maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) { - if (markerChecker != null) { - markerChecker.shutdown(); + if (asyncDetectorExecutor != null) { + asyncDetectorExecutor.shutdown(); } hasConflict.set(false); - markerChecker = Executors.newSingleThreadScheduledExecutor(); - markerChecker.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, - fileSystem, maxAllowableHeartbeatIntervalInMs, oldInstants, checkCommitConflict), + asyncDetectorExecutor = Executors.newSingleThreadScheduledExecutor(); + asyncDetectorExecutor.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, + fileSystem, maxAllowableHeartbeatIntervalInMs, oldInstants, checkCommitConflict), batchInterval, period, TimeUnit.MILLISECONDS); } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index 810be06b4628..51588c73a265 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -18,9 +18,6 @@ package org.apache.hudi.timeline.service.handlers.marker; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -29,6 +26,10 @@ import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.timeline.service.handlers.MarkerHandler; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -101,7 +102,7 @@ public void run() { + "Table markers: " + tableMarkers); hasConflict.compareAndSet(false, true); } - LOG.info("Finish batch marker checker in " + timer.endTimer() + " ms"); + LOG.info("Finish batching marker-based conflict detection in " + timer.endTimer() + " ms"); } catch (IOException e) { throw new HoodieIOException("IOException occurs during checking marker conflict"); diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java index 9ba5ae2af1ef..d0dd9909ed62 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java @@ -18,14 +18,15 @@ package org.apache.hudi.timeline.service.handlers.marker; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.timeline.service.handlers.MarkerHandler; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.jupiter.api.AfterEach; @@ -67,15 +68,15 @@ public void tearDown() throws Exception { /** * Prepare dataset : * $base_path/.hoodie/.temp/2016/001/MARKERS0 => 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0_4-17-21_001.parquet.marker.CREATE (same) - * 2016/4a266542-c7d5-426f-8fb8-fb85a2e88448-0_3-17-20_001.parquet.marker.CREATE - * /002/MARKERS0 => 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0_40-170-210_002.parquet.marker.MERGE (same) - * => 2016/1228caeb-4188-4e19-a18d-848e6f9b0448-0_55-55-425_002.parquet.marker.MERGE - * - * - * Run MarkerCheckerRunnable and find there is a conflict 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0 + * 2016/4a266542-c7d5-426f-8fb8-fb85a2e88448-0_3-17-20_001.parquet.marker.CREATE + * /002/MARKERS0 => 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0_40-170-210_002.parquet.marker.MERGE (same) + * => 2016/1228caeb-4188-4e19-a18d-848e6f9b0448-0_55-55-425_002.parquet.marker.MERGE + *

+ *

+ * Run MarkerBasedEarlyConflictDetectionRunnable and find there is a conflict 2016/b21adfa2-7013-4452-a565-4cc39fea5b73-0 */ @Test - public void testMarkerCheckerRunnable() throws IOException, InterruptedException { + public void testMarkerConflictDetectionRunnable() throws IOException, InterruptedException { AtomicBoolean hasConflict = new AtomicBoolean(false); FileSystem fs = new Path(basePath).getFileSystem(new Configuration()); @@ -100,12 +101,12 @@ public void testMarkerCheckerRunnable() throws IOException, InterruptedException oldInstants.add(new HoodieInstant(false, "commit", oldInstant)); when(markerHandler.getAllMarkers(currentMarkerDir)).thenReturn(currentMarkers); - ScheduledExecutorService markerChecker = Executors.newSingleThreadScheduledExecutor(); - markerChecker.submit(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, markerHandler, currentMarkerDir, + ScheduledExecutorService detectorExecutor = Executors.newSingleThreadScheduledExecutor(); + detectorExecutor.submit(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, markerHandler, currentMarkerDir, basePath, fs, Long.MAX_VALUE, oldInstants, true)); - markerChecker.shutdown(); - markerChecker.awaitTermination(60, TimeUnit.SECONDS); + detectorExecutor.shutdown(); + detectorExecutor.awaitTermination(60, TimeUnit.SECONDS); assertTrue(hasConflict.get()); } From 6d19d032600aef9b3825d68a45a7fa5374aab490 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 10 Jan 2023 00:47:17 +0800 Subject: [PATCH 39/51] address comments --- .../org/apache/hudi/io/HoodieWriteHandle.java | 2 +- .../hudi/table/marker/DirectWriteMarkers.java | 7 ++-- ...erBasedEarlyConflictDetectionStrategy.java | 19 +++++++--- ...erBasedEarlyConflictDetectionStrategy.java | 7 ++-- .../TimelineServerBasedWriteMarkers.java | 2 +- .../hudi/table/marker/WriteMarkers.java | 35 +++++++------------ ...erBasedEarlyConflictDetectionStrategy.java | 19 +++------- 7 files changed, 38 insertions(+), 53 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 80b25157b7c1..a4565e6c7a49 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -189,7 +189,7 @@ protected Path makeNewFilePath(String partitionPath, String fileName) { */ protected void createMarkerFile(String partitionPath, String dataFileName) { WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime) - .create(partitionPath, dataFileName, getIOType(), Option.of(this)); + .create(partitionPath, dataFileName, getIOType(), Option.of(config), Option.of(fileId), Option.of(hoodieTable.getMetaClient())); } public Schema getWriterSchemaWithMetaFields() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index 10d013088854..d792f3d32e23 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -48,6 +48,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; /** * Marker operations of directly accessing the file system to create and delete @@ -161,14 +162,12 @@ protected Option create(String partitionPath, String dataFileName, IOType } @Override - public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, + public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { - long maxAllowableHeartbeatIntervalInMs = config.getHoodieClientHeartbeatIntervalInMs() * config.getHoodieClientHeartbeatTolerableMisses(); - HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), - basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, config.earlyConflictDetectionCheckCommitConflict(), maxAllowableHeartbeatIntervalInMs, completedCommitInstants); + fs, partitionPath, fileId, instantTime, activeTimeline, config); strategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index a850ed2f2d28..2a2783587671 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.util.ConcurrentModificationException; import java.util.HashSet; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -42,17 +43,25 @@ public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); + private final String basePath; + private final boolean checkCommitConflict; + private final Set completedCommitInstants; + private final long maxAllowableHeartbeatIntervalInMs; + + public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { + super(fs, partitionPath, fileId, instantTime, activeTimeline, config); + this.basePath = config.getBasePath(); + this.checkCommitConflict = config.earlyConflictDetectionCheckCommitConflict(); + this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + this.maxAllowableHeartbeatIntervalInMs = config.getHoodieClientHeartbeatIntervalInMs() * config.getHoodieClientHeartbeatTolerableMisses(); - public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs, - HashSet completedCommitInstants) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs, completedCommitInstants); } @Override public boolean hasMarkerConflict() { try { - return checkMarkerConflict(activeTimeline, basePath, partitionPath, fileId, fs, instantTime) + return checkMarkerConflict(basePath, maxAllowableHeartbeatIntervalInMs) || (checkCommitConflict && MarkerUtils.hasCommitConflict(activeTimeline, Stream.of(fileId).collect(Collectors.toSet()), completedCommitInstants)); } catch (IOException e) { LOG.warn("Exception occurs during create marker file in eager conflict detection mode."); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index bfd60fe3ddac..2fe9d71e6261 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -36,10 +36,9 @@ public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy ex private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); - public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config, Boolean checkCommitConflict, - Long maxAllowableHeartbeatIntervalInMs, HashSet completedCommitInstants) { - super(basePath, fs, partitionPath, fileId, instantTime, activeTimeline, config, checkCommitConflict, maxAllowableHeartbeatIntervalInMs, completedCommitInstants); + public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { + super(fs, partitionPath, fileId, instantTime, activeTimeline, config); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 7fc76d732a0e..0a39850c28c3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -150,7 +150,7 @@ protected Option create(String partitionPath, String dataFileName, IOType } @Override - public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, + public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 99d17d3cfdc8..132f9018d5b7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -23,20 +23,17 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.Path; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieWriteHandle; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.io.Serializable; import java.util.Set; -import java.util.stream.Collectors; /** * Operates on markers for a given write action (commit, delta commit, compaction). @@ -59,7 +56,7 @@ public WriteMarkers(String basePath, String markerFolderPath, String instantTime } public Option create(String partitionPath, String dataFileName, IOType type) { - return create(partitionPath, dataFileName, type, Option.empty()); + return create(partitionPath, dataFileName, type, false); } /** @@ -68,35 +65,27 @@ public Option create(String partitionPath, String dataFileName, IOType typ * @param partitionPath partition path in the table * @param dataFileName data file name * @param type write IO type - * @param handler could be empty * @return the marker path */ - public Option create(String partitionPath, String dataFileName, IOType type, Option handler) { - boolean checkIfExists = false; + public Option create(String partitionPath, String dataFileName, IOType type, Option writeConfig, + Option fileId, Option metaClient) { + if (writeConfig.isPresent() && fileId.isPresent() && metaClient.isPresent() + && writeConfig.get().getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && writeConfig.get().isEarlyConflictDetectionEnable()) { - if (handler.isPresent() - && handler.get().getConfig().getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() - && handler.get().getConfig().isEarlyConflictDetectionEnable()) { - - HoodieTableMetaClient metaClient = handler.get().getHoodieTableMetaClient(); - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + HoodieActiveTimeline activeTimeline = metaClient.get().getActiveTimeline(); HoodieTimeline pendingCompactionTimeline = activeTimeline.filterPendingCompactionTimeline(); HoodieTimeline pendingReplaceTimeline = activeTimeline.filterPendingReplaceTimeline(); - // TODO if current is compact or clustering then create marker directly without early conflict detection. + // TODO If current is compact or clustering then create marker directly without early conflict detection. // Need to support early conflict detection between table service and common writers. if (pendingCompactionTimeline.containsInstant(instantTime) || pendingReplaceTimeline.containsInstant(instantTime)) { - return create(partitionPath, dataFileName, type, checkIfExists); + return create(partitionPath, dataFileName, type, false); } - Set completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); - - String fileId = handler.get().getFileId(); - HoodieWriteConfig config = handler.get().getConfig(); - return createWithEarlyConflictDetection(partitionPath, dataFileName, type, checkIfExists, completedCommitInstants, config, fileId, activeTimeline); + return createWithEarlyConflictDetection(partitionPath, dataFileName, type, false, writeConfig.get(), fileId.get(), activeTimeline); } else { - // create marker directly - return create(partitionPath, dataFileName, type, checkIfExists); + return create(partitionPath, dataFileName, type, false); } } @@ -211,6 +200,6 @@ protected Path getMarkerPath(String partitionPath, String dataFileName, IOType t * @param checkIfExists whether to check if the marker already exists * @return the marker path or empty option if already exists and {@code checkIfExists} is true */ - public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, Set completedCommitInstants, + public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 32ccb2051722..03639c1d3a5c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -43,30 +43,21 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { private static final Logger LOG = LogManager.getLogger(HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.class); - protected final String basePath; protected final FileSystem fs; protected final String partitionPath; protected final String fileId; protected final String instantTime; protected final HoodieActiveTimeline activeTimeline; protected final HoodieConfig config; - protected Set completedCommitInstants; - protected final Boolean checkCommitConflict; - protected final Long maxAllowableHeartbeatIntervalInMs; - public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieConfig config, Boolean checkCommitConflict, Long maxAllowableHeartbeatIntervalInMs, - HashSet completedCommitInstants) { - this.basePath = basePath; + public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieConfig config) { this.fs = fs; this.partitionPath = partitionPath; this.fileId = fileId; this.instantTime = instantTime; - this.completedCommitInstants = completedCommitInstants; this.activeTimeline = activeTimeline; this.config = config; - this.checkCommitConflict = checkCommitConflict; - this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; } /** @@ -74,13 +65,11 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(String basePath, Ho * In order to reduce the list pressure as much as possible, first we build path prefix in advance: '$base_path/.temp/instant_time/partition_path', * and only list these specific partition_paths we need instead of list all the '$base_path/.temp/' * @param basePath - * @param partitionPath - * @param fileId 162b13d7-9530-48cf-88a4-02241817ae0c-0_1-74-100_003.parquet + * @param maxAllowableHeartbeatIntervalInMs * @return true if current fileID is already existed under .temp/instant_time/partition_path/.. * @throws IOException */ - public boolean checkMarkerConflict(HoodieActiveTimeline activeTimeline, String basePath, String partitionPath, String fileId, - FileSystem fs, String instantTime) throws IOException { + public boolean checkMarkerConflict(String basePath, long maxAllowableHeartbeatIntervalInMs) throws IOException { String tempFolderPath = basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; List candidateInstants = MarkerUtils.getCandidateInstants(activeTimeline, Arrays.stream(fs.listStatus(new Path(tempFolderPath))).map(FileStatus::getPath).collect(Collectors.toList()), From 3f2118a54d327d0b77483b5881dba5ed773bd227 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 10 Jan 2023 01:46:32 +0800 Subject: [PATCH 40/51] address comments --- .../client/transaction/DirectMarkerTransactionManager.java | 2 +- .../org/apache/hudi/table/marker/DirectWriteMarkers.java | 1 - ...mpleDirectMarkerBasedEarlyConflictDetectionStrategy.java | 2 +- ...odieDirectMarkerBasedEarlyConflictDetectionStrategy.java | 4 +--- .../main/java/org/apache/hudi/common/util/MarkerUtils.java | 6 ++---- 5 files changed, 5 insertions(+), 10 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java index 0f5612f414e8..9a4b9e78b0b0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -68,7 +68,7 @@ public void endTransaction(String currentTxnOwnerInstantTime) { */ private static TypedProperties createUpdatedLockProps( HoodieWriteConfig writeConfig, String partitionPath, String fileId) { - if (ZookeeperBasedLockProvider.class.getName().equals(writeConfig.getLockProviderClass())) { + if (!ZookeeperBasedLockProvider.class.getName().equals(writeConfig.getLockProviderClass())) { throw new HoodieNotSupportedException("Only Support ZK-based lock for DirectMarkerTransactionManager now."); } TypedProperties props = new TypedProperties(writeConfig.getProps()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index d792f3d32e23..1e15b0009f7e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -48,7 +48,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.stream.Collectors; /** * Marker operations of directly accessing the file system to create and delete diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java index 2a2783587671..931694aede1b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -53,7 +53,7 @@ public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSy super(fs, partitionPath, fileId, instantTime, activeTimeline, config); this.basePath = config.getBasePath(); this.checkCommitConflict = config.earlyConflictDetectionCheckCommitConflict(); - this.completedCommitInstants = activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + this.completedCommitInstants = new HashSet<>(activeTimeline.getCommitsTimeline().filterCompletedInstants().getInstants()); this.maxAllowableHeartbeatIntervalInMs = config.getHoodieClientHeartbeatIntervalInMs() * config.getHoodieClientHeartbeatTolerableMisses(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java index 03639c1d3a5c..ada530a0179f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; @@ -34,15 +33,14 @@ import java.io.IOException; import java.util.Arrays; -import java.util.HashSet; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { private static final Logger LOG = LogManager.getLogger(HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.class); + protected final FileSystem fs; protected final String partitionPath; protected final String fileId; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 9b24a93e2de2..989950e5ef2e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -266,13 +266,11 @@ public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set public static List getCandidateInstants(HoodieActiveTimeline activeTimeline, List instants, String currentInstantTime, long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) { - HoodieActiveTimeline reloadActive = activeTimeline.reload(); - return instants.stream().map(Path::toString).filter(instantPath -> { String instantTime = markerDirToInstantTime(instantPath); return instantTime.compareToIgnoreCase(currentInstantTime) < 0 - && !reloadActive.filterPendingCompactionTimeline().containsInstant(instantTime) - && !reloadActive.filterPendingReplaceTimeline().containsInstant(instantTime); + && !activeTimeline.filterPendingCompactionTimeline().containsInstant(instantTime) + && !activeTimeline.filterPendingReplaceTimeline().containsInstant(instantTime); }).filter(instantPath -> { try { return !isHeartbeatExpired(markerDirToInstantTime(instantPath), maxAllowableHeartbeatIntervalInMs, fs, basePath); From be0d5b4e5a211b1ec9f3bdad65177e4667f9a222 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 10 Jan 2023 02:02:58 +0800 Subject: [PATCH 41/51] address comments --- .../java/org/apache/hudi/table/marker/DirectWriteMarkers.java | 1 - ...sactionDirectMarkerBasedEarlyConflictDetectionStrategy.java | 3 --- .../hudi/table/marker/TimelineServerBasedWriteMarkers.java | 1 - 3 files changed, 5 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index 1e15b0009f7e..6d3fbafb6ebb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java index ac0a98e4be91..d1416f898e8b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java @@ -21,14 +21,11 @@ import org.apache.hudi.client.transaction.DirectMarkerTransactionManager; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.util.HashSet; - /** * This strategy is used for direct marker writers, trying to do early conflict detection. * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 39b05586524d..187e19877b20 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; From aad218a872b359f5fffeeca2772ff1e95c74c5dc Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 10 Jan 2023 09:06:17 +0800 Subject: [PATCH 42/51] address comments --- .../service/handlers/MarkerHandler.java | 26 ++++++++++++------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 67e4a611f339..e72de50c90a4 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.timeline.service.TimelineService; import org.apache.hudi.timeline.service.handlers.marker.MarkerCreationDispatchingRunnable; @@ -198,16 +199,12 @@ public CompletableFuture createMarker(Context context, String markerDir, earlyConflictDetectionStrategy.detectAndResolveConflictIfNecessary(); + } catch (HoodieEarlyConflictDetectionException ex) { + LOG.warn("Detect write conflict, failed to create marker with early conflict detection enable", ex); + return finishCreateMarkerFuture(context, markerDir, markerName); } catch (Exception ex) { - LOG.warn("Failed to create marker with early conflict detection enable", ex); - MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName); - try { - future.complete(jsonifyResult( - future.getContext(), future.isSuccessful(), metricsRegistry, new ObjectMapper(), LOG)); - } catch (JsonProcessingException e) { - throw new HoodieException("Failed to JSON encode the value", e); - } - return future; + LOG.warn("Catch exception during detect and resolve write conflict."); + return finishCreateMarkerFuture(context, markerDir, markerName); } } @@ -230,6 +227,17 @@ public CompletableFuture createMarker(Context context, String markerDir, return future; } + private CompletableFuture finishCreateMarkerFuture(Context context, String markerDir, String markerName) { + MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName); + try { + future.complete(jsonifyResult( + future.getContext(), future.isSuccessful(), metricsRegistry, new ObjectMapper(), LOG)); + } catch (JsonProcessingException e) { + throw new HoodieException("Failed to JSON encode the value", e); + } + return future; + } + /** * Deletes markers in the directory. * From 1b837ec65943331f6c805f723b088240ae66dfaf Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 10 Jan 2023 09:06:33 +0800 Subject: [PATCH 43/51] address comments --- .../apache/hudi/timeline/service/handlers/MarkerHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index e72de50c90a4..7862e7f7f4c2 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -199,8 +199,8 @@ public CompletableFuture createMarker(Context context, String markerDir, earlyConflictDetectionStrategy.detectAndResolveConflictIfNecessary(); - } catch (HoodieEarlyConflictDetectionException ex) { - LOG.warn("Detect write conflict, failed to create marker with early conflict detection enable", ex); + } catch (HoodieEarlyConflictDetectionException e) { + LOG.warn("Detect write conflict, failed to create marker with early conflict detection enable", e); return finishCreateMarkerFuture(context, markerDir, markerName); } catch (Exception ex) { LOG.warn("Catch exception during detect and resolve write conflict."); From c34fb5269b5467498e1fe61d66844b9a9a25a7bf Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 18 Jan 2023 16:50:13 -0800 Subject: [PATCH 44/51] Address review comments --- .../org/apache/hudi/io/HoodieWriteHandle.java | 11 ++++++----- .../apache/hudi/table/marker/WriteMarkers.java | 18 +++++++----------- .../service/handlers/MarkerHandler.java | 18 +++++++++++++----- 3 files changed, 26 insertions(+), 21 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 7b5f9bf954a1..7b68c6c34fe0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -18,10 +18,6 @@ package org.apache.hudi.io; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; @@ -43,6 +39,11 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkersFactory; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -135,7 +136,7 @@ protected Path makeNewFilePath(String partitionPath, String fileName) { */ protected void createMarkerFile(String partitionPath, String dataFileName) { WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime) - .create(partitionPath, dataFileName, getIOType(), Option.of(config), Option.of(fileId), Option.of(hoodieTable.getMetaClient())); + .create(partitionPath, dataFileName, getIOType(), config, fileId); } public Schema getWriterSchemaWithMetaFields() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 082ebff72ec0..1e5e85cceb01 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -63,18 +63,14 @@ public Option create(String partitionPath, String dataFileName, IOType typ * Creates a marker without checking if the marker already exists. * * @param partitionPath partition path in the table - * @param dataFileName data file name - * @param type write IO type + * @param dataFileName data file name + * @param type write IO type * @return the marker path */ - public Option create(String partitionPath, String dataFileName, IOType type, Option writeConfig, - Option fileId, Option metaClient) { - if (writeConfig.isPresent() && fileId.isPresent() && metaClient.isPresent() - && writeConfig.get().getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() - && writeConfig.get().isEarlyConflictDetectionEnable()) { - - HoodieActiveTimeline activeTimeline = metaClient.get().getActiveTimeline(); - + public Option create(String partitionPath, String dataFileName, IOType type, HoodieWriteConfig writeConfig, + String fileId, HoodieActiveTimeline activeTimeline) { + if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() + && writeConfig.isEarlyConflictDetectionEnable()) { HoodieTimeline pendingCompactionTimeline = activeTimeline.filterPendingCompactionTimeline(); HoodieTimeline pendingReplaceTimeline = activeTimeline.filterPendingReplaceTimeline(); // TODO If current is compact or clustering then create marker directly without early conflict detection. @@ -83,7 +79,7 @@ public Option create(String partitionPath, String dataFileName, IOType typ return create(partitionPath, dataFileName, type, false); } - return createWithEarlyConflictDetection(partitionPath, dataFileName, type, false, writeConfig.get(), fileId.get(), activeTimeline); + return createWithEarlyConflictDetection(partitionPath, dataFileName, type, false, writeConfig, fileId, activeTimeline); } else { return create(partitionPath, dataFileName, type, false); } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 7862e7f7f4c2..b5675e80e552 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -199,16 +199,24 @@ public CompletableFuture createMarker(Context context, String markerDir, earlyConflictDetectionStrategy.detectAndResolveConflictIfNecessary(); - } catch (HoodieEarlyConflictDetectionException e) { - LOG.warn("Detect write conflict, failed to create marker with early conflict detection enable", e); - return finishCreateMarkerFuture(context, markerDir, markerName); - } catch (Exception ex) { - LOG.warn("Catch exception during detect and resolve write conflict."); + } catch (HoodieEarlyConflictDetectionException he) { + LOG.warn("Detected the write conflict due to a concurrent writer, " + + "failing the marker creation as the early conflict detection is enabled", he); return finishCreateMarkerFuture(context, markerDir, markerName); + } catch (Exception e) { + LOG.warn("Failed to execute early conflict detection." + e.getMessage()); + // When early conflict detection fails to execute, we still allow the marker creation + // to continue + return addMarkerCreationRequestForAsyncProcessing(context, markerDir, markerName); } } // Step 2 create marker + return addMarkerCreationRequestForAsyncProcessing(context, markerDir, markerName); + } + + private MarkerCreationFuture addMarkerCreationRequestForAsyncProcessing( + Context context, String markerDir, String markerName) { LOG.info("Request: create marker " + markerDir + " " + markerName); MarkerCreationFuture future = new MarkerCreationFuture(context, markerDir, markerName); // Add the future to the list From 67b3892f4bafe64ecf203a53bb86db22e61ae587 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 18 Jan 2023 17:12:20 -0800 Subject: [PATCH 45/51] Fix build --- .../src/main/java/org/apache/hudi/io/HoodieWriteHandle.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 7b68c6c34fe0..8e470471db91 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -136,7 +136,7 @@ protected Path makeNewFilePath(String partitionPath, String fileName) { */ protected void createMarkerFile(String partitionPath, String dataFileName) { WriteMarkersFactory.get(config.getMarkersType(), hoodieTable, instantTime) - .create(partitionPath, dataFileName, getIOType(), config, fileId); + .create(partitionPath, dataFileName, getIOType(), config, fileId, hoodieTable.getMetaClient().getActiveTimeline()); } public Schema getWriterSchemaWithMetaFields() { From a2980b73d4ad32976360804059fcd6df969b9f89 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 19 Jan 2023 19:53:57 -0800 Subject: [PATCH 46/51] Fix diverging changes from master and nits --- .../hudi/table/marker/WriteMarkers.java | 4 +- .../client/TestHoodieClientMultiWriter.java | 40 +++++-------------- .../apache/hudi/common/util/MarkerUtils.java | 5 ++- 3 files changed, 14 insertions(+), 35 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 1e5e85cceb01..713374229e30 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -78,11 +78,9 @@ public Option create(String partitionPath, String dataFileName, IOType typ if (pendingCompactionTimeline.containsInstant(instantTime) || pendingReplaceTimeline.containsInstant(instantTime)) { return create(partitionPath, dataFileName, type, false); } - return createWithEarlyConflictDetection(partitionPath, dataFileName, type, false, writeConfig, fileId, activeTimeline); - } else { - return create(partitionPath, dataFileName, type, false); } + return create(partitionPath, dataFileName, type, false); } /** 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 index 3bf0669e75b6..f6cb7e3d1a65 100644 --- 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 @@ -272,7 +272,7 @@ public void testHoodieClientBasicMultiWriter(HoodieTableType tableType, Class pr .build()).withAutoCommit(false).withProperties(lockProperties).build(); // Create the first commit - createCommitWithBulkInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", "001", 200, true); + createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", "001", 200, true); final int threadCount = 2; final ExecutorService executors = Executors.newFixedThreadPool(2); @@ -434,7 +434,7 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta // Create the first commit with inserts HoodieWriteConfig cfg = writeConfigBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); - createCommitWithBulkInserts(cfg, client, "000", "001", 200, true); + createCommitWithInserts(cfg, client, "000", "001", 200, true); validInstants.add("001"); // Create 2 commits with upserts createCommitWithUpserts(cfg, client, "001", "000", "002", 100); @@ -509,7 +509,7 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta final int numRecords = 100; latchCountDownAndWait(runCountDownLatch, 30000); assertDoesNotThrow(() -> { - createCommitWithBulkInserts(cfg, client1, "003", newCommitTime, numRecords, true); + createCommitWithInserts(cfg, client1, "003", newCommitTime, numRecords, true); validInstants.add("007"); }); }); @@ -570,7 +570,7 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) .build(); // Create the first commit - createCommitWithBulkInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, true); + createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, true); // Start another inflight commit String newCommitTime = "003"; int numRecords = 100; @@ -616,7 +616,7 @@ public void testHoodieClientMultiWriterAutoCommitForConflict() throws Exception HoodieWriteConfig cfg2 = writeConfigBuilder.build(); // Create the first commit - createCommitWithBulkInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 5000, false); + createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 5000, false); // Start another inflight commit String newCommitTime1 = "003"; String newCommitTime2 = "004"; @@ -700,7 +700,7 @@ public void testHoodieClientMultiWriterAutoCommitNonConflict() throws Exception HoodieWriteConfig cfg2 = writeConfigBuilder.build(); // Create the first commit - createCommitWithBulkInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, false); + createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200, false); // Start another inflight commit String newCommitTime1 = "003"; String newCommitTime2 = "004"; @@ -734,7 +734,7 @@ private void createCommitWithInsertsForPartition(HoodieWriteConfig cfg, SparkRDD assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); } - private void createCommitWithBulkInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, + private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommitTime, String newCommitTime, int numRecords, boolean doCommit) throws Exception { // Finish first base commit @@ -745,33 +745,13 @@ private void createCommitWithBulkInserts(HoodieWriteConfig cfg, SparkRDDWriteCli } } - private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, - String prevCommitTime, String newCommitTime, int numRecords, - boolean doCommit) throws Exception { - // Finish first base commit - JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::insert, - false, false, numRecords); - if (doCommit) { - assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); - } - } - - private JavaRDD createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommit, - String commitTimeBetweenPrevAndNew, String newCommitTime, int numRecords) throws Exception { - return createCommitWithUpserts(cfg, client, prevCommit, commitTimeBetweenPrevAndNew, newCommitTime, numRecords, true); - } - - private JavaRDD createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommit, - String commitTimeBetweenPrevAndNew, String newCommitTime, int numRecords, boolean commit) + 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); - if (commit) { - client.commit(newCommitTime, result); - } - - return result; + client.commit(newCommitTime, result); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 989950e5ef2e..9a411ec2c872 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -220,10 +220,11 @@ public static Set readMarkersFromFile(Path markersFilePath, Serializable fsDataInputStream = fs.open(markersFilePath); markers = new HashSet<>(FileIOUtils.readAsUTFStringLines(fsDataInputStream)); } catch (IOException e) { + String errorMessage = "Failed to read MARKERS file " + markersFilePath; if (ignoreException) { - LOG.warn("IOException occurs during read MARKERS file, ", e); + LOG.warn(errorMessage + ". Ignoring the exception and continue.", e); } else { - throw new HoodieIOException("Failed to read MARKERS file " + markersFilePath, e); + throw new HoodieIOException(errorMessage, e); } } finally { closeQuietly(fsDataInputStream); From 0579f9b8271b238fa60b20eadb1772689d7aad06 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 19 Jan 2023 19:55:27 -0800 Subject: [PATCH 47/51] Fix config inference --- .../src/main/java/org/apache/hudi/config/HoodieWriteConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4a9426e9cde0..8b9ff0d97e96 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 @@ -559,7 +559,7 @@ public class HoodieWriteConfig extends HoodieConfig { .noDefaultValue() .sinceVersion("0.13.0") .withInferFunction(cfg -> { - MarkerType markerType = MarkerType.valueOf(cfg.getString(MARKERS_TYPE).toUpperCase()); + MarkerType markerType = MarkerType.valueOf(cfg.getStringOrDefault(MARKERS_TYPE).toUpperCase()); switch (markerType) { case DIRECT: return Option.of(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()); From 29761671b4ecf1d01b50349093e0bb054c5ff2bd Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 19 Jan 2023 20:50:05 -0800 Subject: [PATCH 48/51] Add and revise javadocs, make strategy class names shorter --- .../DirectMarkerTransactionManager.java | 13 +++- .../apache/hudi/config/HoodieWriteConfig.java | 9 ++- .../hudi/table/marker/DirectWriteMarkers.java | 8 +-- ...leDirectMarkerBasedDetectionStrategy.java} | 14 +++-- ...onDirectMarkerBasedDetectionStrategy.java} | 14 +++-- .../TimelineServerBasedWriteMarkers.java | 26 ++++---- .../hudi/table/marker/WriteMarkers.java | 36 ++++++++--- .../client/TestHoodieClientMultiWriter.java | 18 +++--- ...> DirectMarkerBasedDetectionStrategy.java} | 30 +++++---- ...va => EarlyConflictDetectionStrategy.java} | 18 +++++- ...erBasedEarlyConflictDetectionStrategy.java | 42 ------------- .../TimelineServerBasedDetectionStrategy.java | 61 +++++++++++++++++++ .../heartbeat/HoodieHeartbeatUtils.java | 20 ++++-- .../apache/hudi/common/util/MarkerUtils.java | 24 ++++++++ .../service/handlers/MarkerHandler.java | 16 ++--- ...AsyncTimelineMarkerDetectionStrategy.java} | 28 ++++++--- ...erBasedEarlyConflictDetectionRunnable.java | 8 +-- ...erBasedEarlyConflictDetectionRunnable.java | 3 + 18 files changed, 253 insertions(+), 135 deletions(-) rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/{SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java => SimpleDirectMarkerBasedDetectionStrategy.java} (84%) rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/{SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java => SimpleTransactionDirectMarkerBasedDetectionStrategy.java} (77%) rename hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/{HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java => DirectMarkerBasedDetectionStrategy.java} (80%) rename hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/{HoodieEarlyConflictDetectionStrategy.java => EarlyConflictDetectionStrategy.java} (68%) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java rename hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/{AsyncTimelineMarkerEarlyConflictDetectionStrategy.java => AsyncTimelineMarkerDetectionStrategy.java} (65%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java index 9a4b9e78b0b0..a6d154bafbbc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -32,6 +32,12 @@ import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; +/** + * This class allows clients to start and end transactions for creating direct marker, used by + * `SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy`, when early conflict + * detection is enabled. Anything done between a start and end transaction is guaranteed to be + * atomic. + */ public class DirectMarkerTransactionManager extends TransactionManager { private final String filePath; @@ -64,7 +70,12 @@ public void endTransaction(String currentTxnOwnerInstantTime) { } /** - * Rebuild lock related configs, only support ZK related lock for now. + * Rebuilds lock related configs. Only support ZK related lock for now. + * + * @param writeConfig Hudi write configs. + * @param partitionPath Relative partition path. + * @param fileId File ID. + * @return Updated lock related configs. */ private static TypedProperties createUpdatedLockProps( HoodieWriteConfig writeConfig, String partitionPath, String fileId) { 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 8b9ff0d97e96..2f2d4a614917 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 @@ -73,9 +73,9 @@ import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; -import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.table.marker.SimpleDirectMarkerBasedDetectionStrategy; import org.apache.hudi.table.storage.HoodieStorageLayout; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerDetectionStrategy; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.log4j.LogManager; @@ -553,7 +553,6 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("When table is upgraded from pre 0.12 to 0.12, we check for \"default\" partition and fail if found one. " + "Users are expected to rewrite the data in those partitions. Enabling this config will bypass this validation"); - // Pluggable strategies to use when early conflict detection public static final ConfigProperty EARLY_CONFLICT_DETECTION_STRATEGY_CLASS_NAME = ConfigProperty .key(CONCURRENCY_PREFIX + "early.conflict.detection.strategy") .noDefaultValue() @@ -562,10 +561,10 @@ public class HoodieWriteConfig extends HoodieConfig { MarkerType markerType = MarkerType.valueOf(cfg.getStringOrDefault(MARKERS_TYPE).toUpperCase()); switch (markerType) { case DIRECT: - return Option.of(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()); + return Option.of(SimpleDirectMarkerBasedDetectionStrategy.class.getName()); case TIMELINE_SERVER_BASED: default: - return Option.of(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()); + return Option.of(AsyncTimelineMarkerDetectionStrategy.class.getName()); } }) .withDocumentation("The class name of the early conflict detection strategy to use. " diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index 6d3fbafb6ebb..b98f9223de7d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.marker; import org.apache.hudi.common.config.SerializableConfiguration; -import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.DirectMarkerBasedDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; @@ -163,9 +163,9 @@ protected Option create(String partitionPath, String dataFileName, IOType public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { - HoodieDirectMarkerBasedEarlyConflictDetectionStrategy strategy = - (HoodieDirectMarkerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), - fs, partitionPath, fileId, instantTime, activeTimeline, config); + DirectMarkerBasedDetectionStrategy strategy = + (DirectMarkerBasedDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), + fs, partitionPath, fileId, instantTime, activeTimeline, config); strategy.detectAndResolveConflictIfNecessary(); return create(getMarkerPath(partitionPath, dataFileName, type), checkIfExists); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java similarity index 84% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java index 931694aede1b..89389ee15a65 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.marker; -import org.apache.hudi.common.conflict.detection.HoodieDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.DirectMarkerBasedDetectionStrategy; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -26,6 +26,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieIOException; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -38,18 +39,19 @@ /** * This strategy is used for direct marker writers, trying to do early conflict detection. - * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. + * It will use fileSystem api like list and exist directly to check if there is any marker file + * conflict, without any locking. */ -public class SimpleDirectMarkerBasedEarlyConflictDetectionStrategy extends HoodieDirectMarkerBasedEarlyConflictDetectionStrategy { +public class SimpleDirectMarkerBasedDetectionStrategy extends DirectMarkerBasedDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class); + private static final Logger LOG = LogManager.getLogger(SimpleDirectMarkerBasedDetectionStrategy.class); private final String basePath; private final boolean checkCommitConflict; private final Set completedCommitInstants; private final long maxAllowableHeartbeatIntervalInMs; - public SimpleDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { + public SimpleDirectMarkerBasedDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { super(fs, partitionPath, fileId, instantTime, activeTimeline, config); this.basePath = config.getBasePath(); this.checkCommitConflict = config.earlyConflictDetectionCheckCommitConflict(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java similarity index 77% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java index d1416f898e8b..00e5779871de 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java @@ -28,14 +28,18 @@ /** * This strategy is used for direct marker writers, trying to do early conflict detection. - * It will use fileSystem api like list and exist directly to check if there is any marker file conflict. + * It will use fileSystem api like list and exist directly to check if there is any marker file + * conflict, with transaction locks using {@link DirectMarkerTransactionManager}. */ -public class SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy extends SimpleDirectMarkerBasedEarlyConflictDetectionStrategy { +public class SimpleTransactionDirectMarkerBasedDetectionStrategy + extends SimpleDirectMarkerBasedDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class); + private static final Logger LOG = LogManager.getLogger( + SimpleTransactionDirectMarkerBasedDetectionStrategy.class); - public SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { + public SimpleTransactionDirectMarkerBasedDetectionStrategy( + HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieWriteConfig config) { super(fs, partitionPath, fileId, instantTime, activeTimeline, config); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 187e19877b20..76eb26b03552 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -137,7 +137,7 @@ protected Option create(String partitionPath, String dataFileName, IOType HoodieTimer timer = HoodieTimer.start(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = initConfigMap(partitionPath, markerFileName, false); + Map paramsMap = getConfigMap(partitionPath, markerFileName, false); boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName + " in " + timer.endTimer() + " ms"); @@ -153,7 +153,7 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { HoodieTimer timer = new HoodieTimer().startTimer(); String markerFileName = getMarkerFileName(dataFileName, type); - Map paramsMap = initConfigMap(partitionPath, markerFileName, true); + Map paramsMap = getConfigMap(partitionPath, markerFileName, true); boolean success = executeCreateMarkerRequest(paramsMap, partitionPath, markerFileName); @@ -169,11 +169,12 @@ public Option createWithEarlyConflictDetection(String partitionPath, Strin } /** - * execute create marker request with specific parasMap - * @param paramsMap - * @param partitionPath - * @param markerFileName - * @return + * Executes marker creation request with specific parameters. + * + * @param paramsMap Parameters to be included in the marker request. + * @param partitionPath Relative partition path. + * @param markerFileName Marker file name. + * @return {@code true} if successful; {@code false} otherwise. */ private boolean executeCreateMarkerRequest(Map paramsMap, String partitionPath, String markerFileName) { boolean success; @@ -188,15 +189,14 @@ private boolean executeCreateMarkerRequest(Map paramsMap, String } /** - * init create marker related config maps. + * Gets parameter map for marker creation request. * - * @param partitionPath - * @param markerFileName - * @return + * @param partitionPath Relative partition path. + * @param markerFileName Marker file name. + * @return parameter map. */ - private Map initConfigMap( + private Map getConfigMap( String partitionPath, String markerFileName, boolean initEarlyConflictDetectionConfigs) { - Map paramsMap = new HashMap<>(); paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); if (StringUtils.isNullOrEmpty(partitionPath)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 713374229e30..17751ad80de1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -55,17 +55,29 @@ public WriteMarkers(String basePath, String markerFolderPath, String instantTime this.instantTime = instantTime; } + /** + * Creates a marker without checking if the marker already exists. + * + * @param partitionPath partition path in the table. + * @param dataFileName data file name. + * @param type write IO type. + * @return the marker path. + */ public Option create(String partitionPath, String dataFileName, IOType type) { return create(partitionPath, dataFileName, type, false); } /** * Creates a marker without checking if the marker already exists. + * This can invoke marker-based early conflict detection when enabled for multi-writers. * - * @param partitionPath partition path in the table - * @param dataFileName data file name - * @param type write IO type - * @return the marker path + * @param partitionPath partition path in the table + * @param dataFileName data file name + * @param type write IO type + * @param writeConfig Hudi write configs. + * @param fileId File ID. + * @param activeTimeline Active timeline for the write operation. + * @return the marker path. */ public Option create(String partitionPath, String dataFileName, IOType type, HoodieWriteConfig writeConfig, String fileId, HoodieActiveTimeline activeTimeline) { @@ -186,13 +198,17 @@ protected Path getMarkerPath(String partitionPath, String dataFileName, IOType t abstract Option create(String partitionPath, String dataFileName, IOType type, boolean checkIfExists); /** - * Creates a marker. + * Creates a marker with early conflict detection for multi-writers. If conflict is detected, + * an exception is thrown to fail the write operation. * - * @param partitionPath partition path in the table - * @param dataFileName data file name - * @param type write IO type - * @param checkIfExists whether to check if the marker already exists - * @return the marker path or empty option if already exists and {@code checkIfExists} is true + * @param partitionPath partition path in the table. + * @param dataFileName data file name. + * @param type write IO type. + * @param checkIfExists whether to check if the marker already exists. + * @param config Hudi write configs. + * @param fileId File ID. + * @param activeTimeline Active timeline for the write operation. + * @return the marker path or empty option if already exists and {@code checkIfExists} is true. */ public abstract Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline); 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 index f6cb7e3d1a65..e0aa4c394c7e 100644 --- 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 @@ -45,10 +45,10 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.marker.SimpleDirectMarkerBasedEarlyConflictDetectionStrategy; -import org.apache.hudi.table.marker.SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.table.marker.SimpleDirectMarkerBasedDetectionStrategy; +import org.apache.hudi.table.marker.SimpleTransactionDirectMarkerBasedDetectionStrategy; import org.apache.hudi.testutils.HoodieClientTestBase; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerDetectionStrategy; import org.apache.curator.test.TestingServer; import org.apache.hadoop.fs.Path; @@ -175,7 +175,7 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta HoodieWriteConfig writeConfig; TestingServer server = null; - if (earlyConflictDetectionStrategy.equalsIgnoreCase(SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName())) { + if (earlyConflictDetectionStrategy.equalsIgnoreCase(SimpleTransactionDirectMarkerBasedDetectionStrategy.class.getName())) { // need to setup zk related env there. Bcz SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy is only support zk lock for now. server = new TestingServer(); Properties properties = new Properties(); @@ -786,11 +786,11 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, public static Stream configParams() { Object[][] data = new Object[][] { - {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerEarlyConflictDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, - {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()}, - {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy.class.getName()} + {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedDetectionStrategy.class.getName()} }; return Stream.of(data).map(Arguments::of); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java similarity index 80% rename from hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java rename to hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java index ada530a0179f..53e91495a901 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java @@ -18,9 +18,6 @@ package org.apache.hudi.common.conflict.detection; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -28,6 +25,10 @@ import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -37,9 +38,12 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { +/** + * This abstract strategy is used for direct marker writers, trying to do early conflict detection. + */ +public abstract class DirectMarkerBasedDetectionStrategy implements EarlyConflictDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(HoodieDirectMarkerBasedEarlyConflictDetectionStrategy.class); + private static final Logger LOG = LogManager.getLogger(DirectMarkerBasedDetectionStrategy.class); protected final FileSystem fs; protected final String partitionPath; @@ -48,8 +52,8 @@ public abstract class HoodieDirectMarkerBasedEarlyConflictDetectionStrategy impl protected final HoodieActiveTimeline activeTimeline; protected final HoodieConfig config; - public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, - HoodieActiveTimeline activeTimeline, HoodieConfig config) { + public DirectMarkerBasedDetectionStrategy(HoodieWrapperFileSystem fs, String partitionPath, String fileId, String instantTime, + HoodieActiveTimeline activeTimeline, HoodieConfig config) { this.fs = fs; this.partitionPath = partitionPath; this.fileId = fileId; @@ -60,12 +64,14 @@ public HoodieDirectMarkerBasedEarlyConflictDetectionStrategy(HoodieWrapperFileSy /** * We need to do list operation here. - * In order to reduce the list pressure as much as possible, first we build path prefix in advance: '$base_path/.temp/instant_time/partition_path', - * and only list these specific partition_paths we need instead of list all the '$base_path/.temp/' - * @param basePath - * @param maxAllowableHeartbeatIntervalInMs + * In order to reduce the list pressure as much as possible, first we build path prefix in advance: + * '$base_path/.temp/instant_time/partition_path', and only list these specific partition_paths + * we need instead of list all the '$base_path/.temp/' + * + * @param basePath Base path of the table. + * @param maxAllowableHeartbeatIntervalInMs Heartbeat timeout. * @return true if current fileID is already existed under .temp/instant_time/partition_path/.. - * @throws IOException + * @throws IOException upon errors. */ public boolean checkMarkerConflict(String basePath, long maxAllowableHeartbeatIntervalInMs) throws IOException { String tempFolderPath = basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java similarity index 68% rename from hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java rename to hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java index 8bbeef146ee8..7ea47f3fc71a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieEarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java @@ -19,13 +19,25 @@ package org.apache.hudi.common.conflict.detection; /** - * + * Interface for pluggable strategy of early conflict detection for multiple writers. */ -interface HoodieEarlyConflictDetectionStrategy { - +interface EarlyConflictDetectionStrategy { + /** + * Detects and resolves the write conflict if necessary. + */ void detectAndResolveConflictIfNecessary(); + /** + * @return whether there's a write conflict based on markers. + */ boolean hasMarkerConflict(); + /** + * Resolves a write conflict. + * + * @param basePath Base path of the table. + * @param partitionPath Relative partition path. + * @param dataFileName Data file name. + */ void resolveMarkerConflict(String basePath, String partitionPath, String dataFileName); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java deleted file mode 100644 index 6ee92fee6c4d..000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/HoodieTimelineServerBasedEarlyConflictDetectionStrategy.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.common.conflict.detection; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hudi.common.table.timeline.HoodieInstant; - -import java.util.Set; - -public abstract class HoodieTimelineServerBasedEarlyConflictDetectionStrategy implements HoodieEarlyConflictDetectionStrategy { - - protected final String basePath; - protected final String markerDir; - protected final String markerName; - protected final boolean checkCommitConflict; - - public HoodieTimelineServerBasedEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { - this.basePath = basePath; - this.markerDir = markerDir; - this.markerName = markerName; - this.checkCommitConflict = checkCommitConflict; - } - - public void fresh(Long batchInterval, Long period, String markerDir, String basePath, - Long maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) {} -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java new file mode 100644 index 000000000000..525660ef308c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java @@ -0,0 +1,61 @@ +/* + * 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.conflict.detection; + +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import org.apache.hadoop.fs.FileSystem; + +import java.util.Set; + +/** + * This abstract strategy is used for writers using timeline-server-based markers, + * trying to do early conflict detection. + */ +public abstract class TimelineServerBasedDetectionStrategy implements EarlyConflictDetectionStrategy { + + protected final String basePath; + protected final String markerDir; + protected final String markerName; + protected final boolean checkCommitConflict; + + public TimelineServerBasedDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { + this.basePath = basePath; + this.markerDir = markerDir; + this.markerName = markerName; + this.checkCommitConflict = checkCommitConflict; + } + + /** + * Starts the async conflict detection thread. + * + * @param batchIntervalMs Batch internal in milliseconds. + * @param periodMs Scheduling period in milliseconds. + * @param markerDir Marker directory. + * @param basePath Base path of the table. + * @param maxAllowableHeartbeatIntervalInMs Heartbeat timeout. + * @param fileSystem {@link FileSystem} instance. + * @param markerHandler Marker handler. + * @param completedCommits Completed Hudi commits. + */ + public abstract void startAsyncDetection(Long batchIntervalMs, Long periodMs, String markerDir, + String basePath, Long maxAllowableHeartbeatIntervalInMs, + FileSystem fileSystem, Object markerHandler, + Set completedCommits); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java index 49c1483756ed..2ef3f4d54dd2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java @@ -35,12 +35,12 @@ public class HoodieHeartbeatUtils { private static final Logger LOG = LogManager.getLogger(HoodieHeartbeatUtils.class); /** - * Use modification time as last heart beat time + * Use modification time as last heart beat time. * - * @param fs - * @param basePath - * @param instantTime - * @return + * @param fs {@link FileSystem} instance. + * @param basePath Base path of the table. + * @param instantTime Instant time. + * @return Last heartbeat timestamp. * @throws IOException */ public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { @@ -53,6 +53,16 @@ public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String i } } + /** + * Whether a heartbeat is expired. + * + * @param instantTime Instant time. + * @param maxAllowableHeartbeatIntervalInMs Heartbeat timeout in milliseconds. + * @param fs {@link FileSystem} instance. + * @param basePath Base path of the table. + * @return {@code true} if expired; {@code false} otherwise. + * @throws IOException upon errors. + */ public static boolean isHeartbeatExpired(String instantTime, long maxAllowableHeartbeatIntervalInMs, FileSystem fs, String basePath) throws IOException { Long currentTime = System.currentTimeMillis(); Long lastHeartbeatTime = getLastHeartbeatTime(fs, basePath, instantTime); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index 9a411ec2c872..0ac8590f0afc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -211,6 +211,14 @@ public static Set readMarkersFromFile(Path markersFilePath, Serializable return readMarkersFromFile(markersFilePath, conf, false); } + /** + * Reads the markers stored in the underlying file. + * + * @param markersFilePath File path for the markers. + * @param conf Serializable config. + * @param ignoreException Whether to ignore IOException. + * @return Markers in a {@code Set} of String. + */ public static Set readMarkersFromFile(Path markersFilePath, SerializableConfiguration conf, boolean ignoreException) { FSDataInputStream fsDataInputStream = null; Set markers = new HashSet<>(); @@ -232,10 +240,26 @@ public static Set readMarkersFromFile(Path markersFilePath, Serializable return markers; } + /** + * Gets all marker directories. + * + * @param tempPath Temporary folder under .hoodie. + * @param fs File system to use. + * @return All marker directories. + * @throws IOException upon error. + */ public static List getAllMarkerDir(Path tempPath, FileSystem fs) throws IOException { return Arrays.stream(fs.listStatus(tempPath)).map(FileStatus::getPath).collect(Collectors.toList()); } + /** + * Whether there is write conflict with completed commit among multiple writers. + * + * @param activeTimeline Active timeline. + * @param currentFileIDs Current set of file IDs. + * @param completedCommitInstants Completed commits. + * @return {@code true} if the conflict is detected; {@code false} otherwise. + */ public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set currentFileIDs, Set completedCommitInstants) { Set currentInstants = new HashSet<>( diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index b5675e80e552..180f72e5f78b 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -18,7 +18,7 @@ package org.apache.hudi.timeline.service.handlers; -import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.TimelineServerBasedDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.IOType; @@ -98,7 +98,7 @@ public class MarkerHandler extends Handler { private ScheduledFuture dispatchingThreadFuture; private boolean firstCreationRequestSeen; private String currentMarkerDir = null; - private HoodieTimelineServerBasedEarlyConflictDetectionStrategy earlyConflictDetectionStrategy; + private TimelineServerBasedDetectionStrategy earlyConflictDetectionStrategy; public MarkerHandler(Configuration conf, TimelineService.Config timelineServiceConfig, HoodieEngineContext hoodieEngineContext, FileSystem fileSystem, @@ -173,7 +173,7 @@ public CompletableFuture createMarker(Context context, String markerDir, try { synchronized (earlyConflictDetectionLock) { if (earlyConflictDetectionStrategy == null) { - earlyConflictDetectionStrategy = (HoodieTimelineServerBasedEarlyConflictDetectionStrategy) ReflectionUtils.loadClass(timelineServiceConfig.earlyConflictDetectionStrategy, + earlyConflictDetectionStrategy = (TimelineServerBasedDetectionStrategy) ReflectionUtils.loadClass(timelineServiceConfig.earlyConflictDetectionStrategy, basePath, markerDir, markerName, timelineServiceConfig.checkCommitConflict); } @@ -184,16 +184,18 @@ public CompletableFuture createMarker(Context context, String markerDir, if (!markerDir.equalsIgnoreCase(currentMarkerDir)) { this.currentMarkerDir = markerDir; Set actions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION); - Set oldInstants = new HashSet<>( + Set completedCommits = new HashSet<>( viewManager.getFileSystemView(basePath) .getTimeline() .filterCompletedInstants() .filter(instant -> actions.contains(instant.getAction())) .getInstants()); - earlyConflictDetectionStrategy.fresh(timelineServiceConfig.asyncConflictDetectorBatchIntervalMs, - timelineServiceConfig.asyncConflictDetectorBatchPeriodMs, markerDir, basePath, timelineServiceConfig.maxAllowableHeartbeatIntervalInMs, fileSystem, - this, oldInstants); + earlyConflictDetectionStrategy.startAsyncDetection( + timelineServiceConfig.asyncConflictDetectorBatchIntervalMs, + timelineServiceConfig.asyncConflictDetectorBatchPeriodMs, + markerDir, basePath, timelineServiceConfig.maxAllowableHeartbeatIntervalInMs, + fileSystem, this, completedCommits); } } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerDetectionStrategy.java similarity index 65% rename from hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java rename to hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerDetectionStrategy.java index 0bf059996060..930ff8a94d50 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerEarlyConflictDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerDetectionStrategy.java @@ -18,7 +18,7 @@ package org.apache.hudi.timeline.service.handlers.marker; -import org.apache.hudi.common.conflict.detection.HoodieTimelineServerBasedEarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.TimelineServerBasedDetectionStrategy; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.timeline.service.handlers.MarkerHandler; @@ -34,14 +34,19 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -public class AsyncTimelineMarkerEarlyConflictDetectionStrategy extends HoodieTimelineServerBasedEarlyConflictDetectionStrategy { +/** + * This abstract strategy is used for writers using timeline-server-based markers, + * trying to do early conflict detection by asynchronously and periodically checking + * write conflict among multiple writers based on the timeline-server-based markers. + */ +public class AsyncTimelineMarkerDetectionStrategy extends TimelineServerBasedDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(AsyncTimelineMarkerEarlyConflictDetectionStrategy.class); + private static final Logger LOG = LogManager.getLogger(AsyncTimelineMarkerDetectionStrategy.class); private AtomicBoolean hasConflict = new AtomicBoolean(false); private ScheduledExecutorService asyncDetectorExecutor; - public AsyncTimelineMarkerEarlyConflictDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { + public AsyncTimelineMarkerDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { super(basePath, markerDir, markerName, checkCommitConflict); } @@ -55,16 +60,21 @@ public void resolveMarkerConflict(String basePath, String markerDir, String mark throw new HoodieEarlyConflictDetectionException(new ConcurrentModificationException("Early conflict detected but cannot resolve conflicts for overlapping writes")); } - public void fresh(Long batchInterval, Long period, String markerDir, String basePath, - Long maxAllowableHeartbeatIntervalInMs, FileSystem fileSystem, Object markerHandler, Set oldInstants) { + @Override + public void startAsyncDetection(Long batchIntervalMs, Long periodMs, String markerDir, + String basePath, Long maxAllowableHeartbeatIntervalInMs, + FileSystem fileSystem, Object markerHandler, + Set completedCommits) { if (asyncDetectorExecutor != null) { asyncDetectorExecutor.shutdown(); } hasConflict.set(false); asyncDetectorExecutor = Executors.newSingleThreadScheduledExecutor(); - asyncDetectorExecutor.scheduleAtFixedRate(new MarkerBasedEarlyConflictDetectionRunnable(hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, - fileSystem, maxAllowableHeartbeatIntervalInMs, oldInstants, checkCommitConflict), - batchInterval, period, TimeUnit.MILLISECONDS); + asyncDetectorExecutor.scheduleAtFixedRate( + new MarkerBasedEarlyConflictDetectionRunnable( + hasConflict, (MarkerHandler) markerHandler, markerDir, basePath, + fileSystem, maxAllowableHeartbeatIntervalInMs, completedCommits, checkCommitConflict), + batchIntervalMs, periodMs, TimeUnit.MILLISECONDS); } @Override diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index 51588c73a265..e790eb19dbdf 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -49,19 +49,19 @@ public class MarkerBasedEarlyConflictDetectionRunnable implements Runnable { private FileSystem fs; private AtomicBoolean hasConflict; private long maxAllowableHeartbeatIntervalInMs; - private Set oldInstants; + private Set completedCommits; private final boolean checkCommitConflict; public MarkerBasedEarlyConflictDetectionRunnable(AtomicBoolean hasConflict, MarkerHandler markerHandler, String markerDir, String basePath, FileSystem fileSystem, long maxAllowableHeartbeatIntervalInMs, - Set oldInstants, boolean checkCommitConflict) { + Set completedCommits, boolean checkCommitConflict) { this.markerHandler = markerHandler; this.markerDir = markerDir; this.basePath = basePath; this.fs = fileSystem; this.hasConflict = hasConflict; this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs; - this.oldInstants = oldInstants; + this.completedCommits = completedCommits; this.checkCommitConflict = checkCommitConflict; } @@ -96,7 +96,7 @@ public void run() { currentFileIDs.retainAll(tableFilesIDs); if (!currentFileIDs.isEmpty() || (checkCommitConflict && MarkerUtils.hasCommitConflict(activeTimeline, - currentInstantAllMarkers.stream().map(MarkerUtils::makerToPartitionAndFileID).collect(Collectors.toSet()), oldInstants))) { + currentInstantAllMarkers.stream().map(MarkerUtils::makerToPartitionAndFileID).collect(Collectors.toSet()), completedCommits))) { LOG.warn("Conflict writing detected based on markers!\n" + "Conflict markers: " + currentInstantAllMarkers + "\n" + "Table markers: " + tableMarkers); diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java index d0dd9909ed62..f5cba27deba4 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java @@ -49,6 +49,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +/** + * Tests {@link MarkerBasedEarlyConflictDetectionRunnable}. + */ public class TestMarkerBasedEarlyConflictDetectionRunnable extends HoodieCommonTestHarness { private static final Logger LOG = LogManager.getLogger(TestMarkerBasedEarlyConflictDetectionRunnable.class); From 7344fabacee437adb4b55ca922df0f5fb14ae372 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 19 Jan 2023 20:58:33 -0800 Subject: [PATCH 49/51] Revise other names --- .../transaction/DirectMarkerTransactionManager.java | 2 +- .../org/apache/hudi/config/HoodieWriteConfig.java | 10 +++++----- .../hudi/client/TestHoodieClientMultiWriter.java | 8 ++++---- .../hudi/timeline/service/TimelineService.java | 12 ++++++------ ...> AsyncTimelineServerBasedDetectionStrategy.java} | 6 +++--- 5 files changed, 19 insertions(+), 19 deletions(-) rename hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/{AsyncTimelineMarkerDetectionStrategy.java => AsyncTimelineServerBasedDetectionStrategy.java} (92%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java index a6d154bafbbc..84fac2db004a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -34,7 +34,7 @@ /** * This class allows clients to start and end transactions for creating direct marker, used by - * `SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy`, when early conflict + * `SimpleTransactionDirectMarkerBasedDetectionStrategy`, when early conflict * detection is enabled. Anything done between a start and end transaction is guaranteed to be * atomic. */ 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 2f2d4a614917..4587fbac7717 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 @@ -75,7 +75,7 @@ import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.marker.SimpleDirectMarkerBasedDetectionStrategy; import org.apache.hudi.table.storage.HoodieStorageLayout; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerDetectionStrategy; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.log4j.LogManager; @@ -564,12 +564,12 @@ public class HoodieWriteConfig extends HoodieConfig { return Option.of(SimpleDirectMarkerBasedDetectionStrategy.class.getName()); case TIMELINE_SERVER_BASED: default: - return Option.of(AsyncTimelineMarkerDetectionStrategy.class.getName()); + return Option.of(AsyncTimelineServerBasedDetectionStrategy.class.getName()); } }) .withDocumentation("The class name of the early conflict detection strategy to use. " + "This should be a subclass of " - + "`org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy`."); + + "`org.apache.hudi.common.conflict.detection.EarlyConflictDetectionStrategy`."); public static final ConfigProperty EARLY_CONFLICT_DETECTION_ENABLE = ConfigProperty .key(CONCURRENCY_PREFIX + "early.conflict.detection.enable") @@ -584,7 +584,7 @@ public class HoodieWriteConfig extends HoodieConfig { .defaultValue(30000L) .sinceVersion("0.13.0") .withDocumentation("Used for timeline-server-based markers with " - + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "`AsyncTimelineServerBasedDetectionStrategy`. " + "The time in milliseconds to delay first async marker conflict detection."); public static final ConfigProperty ASYNC_CONFLICT_DETECTOR_PERIOD_MS = ConfigProperty @@ -592,7 +592,7 @@ public class HoodieWriteConfig extends HoodieConfig { .defaultValue(30000L) .sinceVersion("0.13.0") .withDocumentation("Used for timeline-server-based markers with " - + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "`AsyncTimelineServerBasedDetectionStrategy`. " + "The period in milliseconds between consecutive runs of async marker conflict detection."); public static final ConfigProperty EARLY_CONFLICT_DETECTION_CHECK_COMMIT_CONFLICT = ConfigProperty 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 index e0aa4c394c7e..bb96f6331201 100644 --- 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 @@ -48,7 +48,7 @@ import org.apache.hudi.table.marker.SimpleDirectMarkerBasedDetectionStrategy; import org.apache.hudi.table.marker.SimpleTransactionDirectMarkerBasedDetectionStrategy; import org.apache.hudi.testutils.HoodieClientTestBase; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerDetectionStrategy; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy; import org.apache.curator.test.TestingServer; import org.apache.hadoop.fs.Path; @@ -176,7 +176,7 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta HoodieWriteConfig writeConfig; TestingServer server = null; if (earlyConflictDetectionStrategy.equalsIgnoreCase(SimpleTransactionDirectMarkerBasedDetectionStrategy.class.getName())) { - // need to setup zk related env there. Bcz SimpleTransactionDirectMarkerBasedEarlyConflictDetectionStrategy is only support zk lock for now. + // need to setup zk related env there. Bcz SimpleTransactionDirectMarkerBasedDetectionStrategy is only support zk lock for now. server = new TestingServer(); Properties properties = new Properties(); properties.setProperty(ZK_BASE_PATH_PROP_KEY, basePath); @@ -786,8 +786,8 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, public static Stream configParams() { Object[][] data = new Object[][] { - {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineMarkerDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineServerBasedDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineServerBasedDetectionStrategy.class.getName()}, {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedDetectionStrategy.class.getName()} diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index e3737d09d60a..2bf3f4dea500 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -127,8 +127,8 @@ public static class Config implements Serializable { @Parameter(names = {"--early-conflict-detection-strategy"}, description = "The class name of the early conflict detection strategy to use. " + "This should be subclass of " - + "`org.apache.hudi.common.conflict.detection.HoodieEarlyConflictDetectionStrategy`") - public String earlyConflictDetectionStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; + + "`org.apache.hudi.common.conflict.detection.EarlyConflictDetectionStrategy`") + public String earlyConflictDetectionStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy"; @Parameter(names = {"--early-conflict-detection-check-commit-conflict"}, description = "Whether to enable commit conflict checking or not during early " @@ -143,19 +143,19 @@ public static class Config implements Serializable { @Parameter(names = {"--async-conflict-detector-batch-interval-ms"}, description = "Used for timeline-server-based markers with " - + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "`AsyncTimelineServerBasedDetectionStrategy`. " + "The time in milliseconds to delay first async marker conflict detection.") public Long asyncConflictDetectorBatchIntervalMs = 30000L; @Parameter(names = {"--async-conflict-detector-batch-period-ms"}, description = "Used for timeline-server-based markers with " - + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "`AsyncTimelineServerBasedDetectionStrategy`. " + "The period in milliseconds between consecutive runs of async marker conflict detection.") public Long asyncConflictDetectorBatchPeriodMs = 30000L; @Parameter(names = {"--early-conflict-detection-max-heartbeat-interval-ms"}, description = "Used for timeline-server-based markers with " - + "`AsyncTimelineMarkerConflictResolutionStrategy`. " + + "`AsyncTimelineServerBasedDetectionStrategy`. " + "Instants whose heartbeat is greater than the current value will not be used in early conflict detection.") public Long maxAllowableHeartbeatIntervalInMs = 60000L; @@ -183,7 +183,7 @@ public static class Builder { private int markerBatchNumThreads = 20; private long markerBatchIntervalMs = 50L; private int markerParallelism = 100; - private String earlyConflictDetectionStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineMarkerEarlyConflictDetectionStrategy"; + private String earlyConflictDetectionStrategy = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy"; private Boolean checkCommitConflict = false; private Boolean earlyConflictDetectionEnable = false; private Long asyncConflictDetectorBatchIntervalMs = 30000L; diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java similarity index 92% rename from hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerDetectionStrategy.java rename to hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java index 930ff8a94d50..1e305376e569 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineMarkerDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java @@ -39,14 +39,14 @@ * trying to do early conflict detection by asynchronously and periodically checking * write conflict among multiple writers based on the timeline-server-based markers. */ -public class AsyncTimelineMarkerDetectionStrategy extends TimelineServerBasedDetectionStrategy { +public class AsyncTimelineServerBasedDetectionStrategy extends TimelineServerBasedDetectionStrategy { - private static final Logger LOG = LogManager.getLogger(AsyncTimelineMarkerDetectionStrategy.class); + private static final Logger LOG = LogManager.getLogger(AsyncTimelineServerBasedDetectionStrategy.class); private AtomicBoolean hasConflict = new AtomicBoolean(false); private ScheduledExecutorService asyncDetectorExecutor; - public AsyncTimelineMarkerDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { + public AsyncTimelineServerBasedDetectionStrategy(String basePath, String markerDir, String markerName, Boolean checkCommitConflict) { super(basePath, markerDir, markerName, checkCommitConflict); } From 501e47fb5c554a32acd7d2b410b5cdcf880f6057 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 20 Jan 2023 13:37:18 -0800 Subject: [PATCH 50/51] Improve async timeline-server-based conflict detection --- ...pleDirectMarkerBasedDetectionStrategy.java | 3 +- ...ionDirectMarkerBasedDetectionStrategy.java | 3 +- .../client/TestHoodieClientMultiWriter.java | 5 ++ .../EarlyConflictDetectionStrategy.java | 4 +- .../service/handlers/MarkerHandler.java | 24 ++++++- ...cTimelineServerBasedDetectionStrategy.java | 2 +- ...erBasedEarlyConflictDetectionRunnable.java | 17 ++++- .../handlers/marker/MarkerDirState.java | 69 ++++++++++++++++--- 8 files changed, 108 insertions(+), 19 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java index 89389ee15a65..954755da35ae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java @@ -77,8 +77,7 @@ public void resolveMarkerConflict(String basePath, String partitionPath, String } @Override - public void detectAndResolveConflictIfNecessary() { - + public void detectAndResolveConflictIfNecessary() throws HoodieEarlyConflictDetectionException { if (hasMarkerConflict()) { resolveMarkerConflict(basePath, partitionPath, fileId); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java index 00e5779871de..d10d74abb01b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -44,7 +45,7 @@ public SimpleTransactionDirectMarkerBasedDetectionStrategy( } @Override - public void detectAndResolveConflictIfNecessary() { + public void detectAndResolveConflictIfNecessary() throws HoodieEarlyConflictDetectionException { DirectMarkerTransactionManager txnManager = new DirectMarkerTransactionManager((HoodieWriteConfig) config, fs, partitionPath, fileId); try { 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 index bb96f6331201..7fa342916e46 100644 --- 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 @@ -830,6 +830,11 @@ private HoodieWriteConfig buildWriteConfigForEarlyConflictDetect(String markerTy .withAutoArchive(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withMarkersType(MarkerType.TIMELINE_SERVER_BASED.name()) + // Set the batch processing interval for marker requests to be larger than + // the running interval of the async conflict detector so that the conflict can + // be detected before the marker requests are processed at the timeline server + // in the test. + .withMarkersTimelineServerBasedBatchIntervalMs(1000) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(lockProvider).build()) .withEarlyConflictDetectionEnable(true) .withEarlyConflictDetectionStrategy(earlyConflictDetectionStrategy) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java index 7ea47f3fc71a..d191535118be 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.conflict.detection; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; + /** * Interface for pluggable strategy of early conflict detection for multiple writers. */ @@ -25,7 +27,7 @@ interface EarlyConflictDetectionStrategy { /** * Detects and resolves the write conflict if necessary. */ - void detectAndResolveConflictIfNecessary(); + void detectAndResolveConflictIfNecessary() throws HoodieEarlyConflictDetectionException; /** * @return whether there's a write conflict based on markers. diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 180f72e5f78b..2bf6a59b11d8 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieException; @@ -42,6 +43,7 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.Collections; import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -137,6 +139,19 @@ public Set getAllMarkers(String markerDir) { return markerDirState.getAllMarkers(); } + /** + * @param markerDir marker directory path. + * @return Pending markers from the requests to process. + */ + public Set getPendingMarkersToProcess(String markerDir) { + if (markerDirStateMap.containsKey(markerDir)) { + MarkerDirState markerDirState = getMarkerDirState(markerDir); + return markerDirState.getPendingMarkerCreationRequests(false).stream() + .map(MarkerCreationFuture::getMarkerName).collect(Collectors.toSet()); + } + return Collections.emptySet(); + } + /** * @param markerDir marker directory path * @return all marker paths of write IO type "CREATE" and "MERGE" @@ -265,8 +280,13 @@ private MarkerDirState getMarkerDirState(String markerDir) { if (markerDirState == null) { synchronized (markerDirStateMap) { if (markerDirStateMap.get(markerDir) == null) { - markerDirState = new MarkerDirState(markerDir, timelineServiceConfig.markerBatchNumThreads, - fileSystem, metricsRegistry, hoodieEngineContext, parallelism); + Option strategy = + timelineServiceConfig.earlyConflictDetectionEnable + && earlyConflictDetectionStrategy != null + ? Option.of(earlyConflictDetectionStrategy) : Option.empty(); + markerDirState = new MarkerDirState( + markerDir, timelineServiceConfig.markerBatchNumThreads, + strategy, fileSystem, metricsRegistry, hoodieEngineContext, parallelism); markerDirStateMap.put(markerDir, markerDirState); } else { markerDirState = markerDirStateMap.get(markerDir); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java index 1e305376e569..1356db0d96ef 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/AsyncTimelineServerBasedDetectionStrategy.java @@ -78,7 +78,7 @@ public void startAsyncDetection(Long batchIntervalMs, Long periodMs, String mark } @Override - public void detectAndResolveConflictIfNecessary() { + public void detectAndResolveConflictIfNecessary() throws HoodieEarlyConflictDetectionException { if (hasMarkerConflict()) { resolveMarkerConflict(basePath, markerDir, markerName); } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index e790eb19dbdf..58fb14bdfc0c 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -67,13 +68,25 @@ public MarkerBasedEarlyConflictDetectionRunnable(AtomicBoolean hasConflict, Mark @Override public void run() { + // If a conflict among multiple writers is already detected, + // there is no need to run the detection again. + if (hasConflict.get()) { + return; + } + try { - if (!fs.exists(new Path(markerDir))) { + Set pendingMarkers = markerHandler.getPendingMarkersToProcess(markerDir); + + if (!fs.exists(new Path(markerDir)) && pendingMarkers.isEmpty()) { return; } HoodieTimer timer = HoodieTimer.start(); - Set currentInstantAllMarkers = markerHandler.getAllMarkers(markerDir); + Set currentInstantAllMarkers = new HashSet<>(); + // We need to check both the markers already written to the storage + // and the markers from the requests pending processing. + currentInstantAllMarkers.addAll(markerHandler.getAllMarkers(markerDir)); + currentInstantAllMarkers.addAll(pendingMarkers); Path tempPath = new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME); List instants = MarkerUtils.getAllMarkerDir(tempPath, fs); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java index f367ec870eb1..cbffea3db903 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java @@ -18,6 +18,7 @@ package org.apache.hudi.timeline.service.handlers.marker; +import org.apache.hudi.common.conflict.detection.TimelineServerBasedDetectionStrategy; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; @@ -25,6 +26,7 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -81,14 +83,18 @@ public class MarkerDirState implements Serializable { private final List markerCreationFutures = new ArrayList<>(); private final int parallelism; private final Object markerCreationProcessingLock = new Object(); + // Early conflict detection strategy if enabled + private final Option conflictDetectionStrategy; private transient HoodieEngineContext hoodieEngineContext; // Last underlying file index used, for finding the next file index // in a round-robin fashion private int lastFileIndexUsed = -1; private boolean isMarkerTypeWritten = false; - public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem, - Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) { + public MarkerDirState(String markerDirPath, int markerBatchNumThreads, + Option conflictDetectionStrategy, + FileSystem fileSystem, Registry metricsRegistry, + HoodieEngineContext hoodieEngineContext, int parallelism) { this.markerDirPath = markerDirPath; this.fileSystem = fileSystem; this.metricsRegistry = metricsRegistry; @@ -96,6 +102,7 @@ public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSyste this.parallelism = parallelism; this.threadUseStatus = Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList()); + this.conflictDetectionStrategy = conflictDetectionStrategy; // Lazy initialization of markers by reading MARKERS* files on the file system syncMarkersFromFileSystem(); } @@ -167,16 +174,26 @@ public void markFileAsAvailable(int fileIndex) { } /** - * @return futures of pending marker creation requests and removes them from the list. + * @return futures of pending marker creation requests and removes them from the list. */ public List fetchPendingMarkerCreationRequests() { + return getPendingMarkerCreationRequests(true); + } + + /** + * @param shouldClear Should clear the internal request list or not. + * @return futures of pending marker creation requests. + */ + public List getPendingMarkerCreationRequests(boolean shouldClear) { List pendingFutures; synchronized (markerCreationFutures) { if (markerCreationFutures.isEmpty()) { return new ArrayList<>(); } pendingFutures = new ArrayList<>(markerCreationFutures); - markerCreationFutures.clear(); + if (shouldClear) { + markerCreationFutures.clear(); + } } return pendingFutures; } @@ -196,16 +213,33 @@ public void processMarkerCreationRequests( LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath + " numRequests=" + pendingMarkerCreationFutures.size() + " fileIndex=" + fileIndex); - + boolean shouldFlushMarkers = false; + synchronized (markerCreationProcessingLock) { for (MarkerCreationFuture future : pendingMarkerCreationFutures) { String markerName = future.getMarkerName(); boolean exists = allMarkers.contains(markerName); if (!exists) { - allMarkers.add(markerName); - StringBuilder stringBuilder = fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384)); - stringBuilder.append(markerName); - stringBuilder.append('\n'); + if (conflictDetectionStrategy.isPresent()) { + try { + conflictDetectionStrategy.get().detectAndResolveConflictIfNecessary(); + } catch (HoodieEarlyConflictDetectionException he) { + LOG.warn("Detected the write conflict due to a concurrent writer, " + + "failing the marker creation as the early conflict detection is enabled", he); + future.setResult(false); + continue; + } catch (Exception e) { + LOG.warn("Failed to execute early conflict detection." + e.getMessage()); + // When early conflict detection fails to execute, we still allow the marker creation + // to continue + addMarkerToMap(fileIndex, markerName); + future.setResult(true); + shouldFlushMarkers = true; + continue; + } + } + addMarkerToMap(fileIndex, markerName); + shouldFlushMarkers = true; } future.setResult(!exists); } @@ -216,7 +250,9 @@ public void processMarkerCreationRequests( isMarkerTypeWritten = true; } } - flushMarkersToFile(fileIndex); + if (shouldFlushMarkers) { + flushMarkersToFile(fileIndex); + } markFileAsAvailable(fileIndex); for (MarkerCreationFuture future : pendingMarkerCreationFutures) { @@ -267,6 +303,19 @@ private void syncMarkersFromFileSystem() { } } + /** + * Adds a new marker to the in-memory map. + * + * @param fileIndex Marker file index number. + * @param markerName Marker name. + */ + private void addMarkerToMap(int fileIndex, String markerName) { + allMarkers.add(markerName); + StringBuilder stringBuilder = fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384)); + stringBuilder.append(markerName); + stringBuilder.append('\n'); + } + /** * Writes marker type, "TIMELINE_SERVER_BASED", to file. */ From 46e80aea4f869c12ba54166d51ed55cfba5f7ded Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sun, 22 Jan 2023 20:33:34 -0800 Subject: [PATCH 51/51] Add validation of conflict detection strategy to be compatible with the marker type and fallback to default --- .../apache/hudi/config/HoodieWriteConfig.java | 11 +---- .../table/marker/ConflictDetectionUtils.java | 42 +++++++++++++++++ .../hudi/table/marker/DirectWriteMarkers.java | 12 ++++- .../DirectMarkerBasedDetectionStrategy.java | 3 ++ .../EarlyConflictDetectionStrategy.java | 5 ++- .../TimelineServerBasedDetectionStrategy.java | 3 ++ .../hudi/common/util/ReflectionUtils.java | 12 +++++ .../hudi/common/util/TestReflectionUtils.java | 45 +++++++++++++++++++ .../service/handlers/MarkerHandler.java | 12 ++++- 9 files changed, 131 insertions(+), 14 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/ConflictDetectionUtils.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/common/util/TestReflectionUtils.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 4587fbac7717..77099e39cabc 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 @@ -73,9 +73,7 @@ import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; -import org.apache.hudi.table.marker.SimpleDirectMarkerBasedDetectionStrategy; import org.apache.hudi.table.storage.HoodieStorageLayout; -import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.log4j.LogManager; @@ -101,6 +99,7 @@ import static org.apache.hudi.common.util.queue.ExecutorType.BOUNDED_IN_MEMORY; import static org.apache.hudi.common.util.queue.ExecutorType.DISRUPTOR; import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY; +import static org.apache.hudi.table.marker.ConflictDetectionUtils.getDefaultEarlyConflictDetectionStrategy; /** * Class storing configs for the HoodieWriteClient. @@ -559,13 +558,7 @@ public class HoodieWriteConfig extends HoodieConfig { .sinceVersion("0.13.0") .withInferFunction(cfg -> { MarkerType markerType = MarkerType.valueOf(cfg.getStringOrDefault(MARKERS_TYPE).toUpperCase()); - switch (markerType) { - case DIRECT: - return Option.of(SimpleDirectMarkerBasedDetectionStrategy.class.getName()); - case TIMELINE_SERVER_BASED: - default: - return Option.of(AsyncTimelineServerBasedDetectionStrategy.class.getName()); - } + return Option.of(getDefaultEarlyConflictDetectionStrategy(markerType)); }) .withDocumentation("The class name of the early conflict detection strategy to use. " + "This should be a subclass of " diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/ConflictDetectionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/ConflictDetectionUtils.java new file mode 100644 index 000000000000..26d8c5b54a24 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/ConflictDetectionUtils.java @@ -0,0 +1,42 @@ +/* + * 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.marker; + +import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy; + +/** + * Utils for early conflict detection. + */ +public class ConflictDetectionUtils { + /** + * @param markerType Marker type. + * @return The class name of the default strategy for early conflict detection. + */ + public static String getDefaultEarlyConflictDetectionStrategy(MarkerType markerType) { + switch (markerType) { + case DIRECT: + return SimpleDirectMarkerBasedDetectionStrategy.class.getName(); + case TIMELINE_SERVER_BASED: + default: + return AsyncTimelineServerBasedDetectionStrategy.class.getName(); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java index b98f9223de7d..6c97e063323c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; @@ -48,6 +49,8 @@ import java.util.List; import java.util.Set; +import static org.apache.hudi.table.marker.ConflictDetectionUtils.getDefaultEarlyConflictDetectionStrategy; + /** * Marker operations of directly accessing the file system to create and delete * marker files. Each data file has a corresponding marker file. @@ -162,9 +165,14 @@ protected Option create(String partitionPath, String dataFileName, IOType @Override public Option createWithEarlyConflictDetection(String partitionPath, String dataFileName, IOType type, boolean checkIfExists, HoodieWriteConfig config, String fileId, HoodieActiveTimeline activeTimeline) { - + String strategyClassName = config.getEarlyConflictDetectionStrategyClassName(); + if (!ReflectionUtils.isSubClass(strategyClassName, DirectMarkerBasedDetectionStrategy.class)) { + LOG.warn("Cannot use " + strategyClassName + " for direct markers."); + strategyClassName = getDefaultEarlyConflictDetectionStrategy(MarkerType.DIRECT); + LOG.warn("Falling back to " + strategyClassName); + } DirectMarkerBasedDetectionStrategy strategy = - (DirectMarkerBasedDetectionStrategy) ReflectionUtils.loadClass(config.getEarlyConflictDetectionStrategyClassName(), + (DirectMarkerBasedDetectionStrategy) ReflectionUtils.loadClass(strategyClassName, fs, partitionPath, fileId, instantTime, activeTimeline, config); strategy.detectAndResolveConflictIfNecessary(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java index 53e91495a901..67286d4f0b3a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.conflict.detection; +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIClass; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -41,6 +43,7 @@ /** * This abstract strategy is used for direct marker writers, trying to do early conflict detection. */ +@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) public abstract class DirectMarkerBasedDetectionStrategy implements EarlyConflictDetectionStrategy { private static final Logger LOG = LogManager.getLogger(DirectMarkerBasedDetectionStrategy.class); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java index d191535118be..ccb97e1e8bf7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/EarlyConflictDetectionStrategy.java @@ -18,12 +18,15 @@ package org.apache.hudi.common.conflict.detection; +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIClass; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; /** * Interface for pluggable strategy of early conflict detection for multiple writers. */ -interface EarlyConflictDetectionStrategy { +@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) +public interface EarlyConflictDetectionStrategy { /** * Detects and resolves the write conflict if necessary. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java index 525660ef308c..51df6216a2dc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/TimelineServerBasedDetectionStrategy.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.conflict.detection; +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIClass; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hadoop.fs.FileSystem; @@ -28,6 +30,7 @@ * This abstract strategy is used for writers using timeline-server-based markers, * trying to do early conflict detection. */ +@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) public abstract class TimelineServerBasedDetectionStrategy implements EarlyConflictDetectionStrategy { protected final String basePath; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index b3e178320b82..92b2864c7611 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -167,4 +167,16 @@ private static List findClasses(File directory, String packageName) { public static boolean isSameClass(Comparable v, Comparable o) { return v.getClass() == o.getClass(); } + + /** + * Checks if the given class with the name is a subclass of another class. + * + * @param aClazzName Class name. + * @param superClazz Super class to check. + * @return {@code true} if {@code aClazzName} is a subclass of {@code superClazz}; + * {@code false} otherwise. + */ + public static boolean isSubClass(String aClazzName, Class superClazz) { + return superClazz.isAssignableFrom(getClass(aClazzName)); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestReflectionUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestReflectionUtils.java new file mode 100644 index 000000000000..15decc3e5c57 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestReflectionUtils.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.hudi.common.conflict.detection.DirectMarkerBasedDetectionStrategy; +import org.apache.hudi.common.conflict.detection.EarlyConflictDetectionStrategy; +import org.apache.hudi.common.conflict.detection.TimelineServerBasedDetectionStrategy; + +import org.junit.jupiter.api.Test; + +import static org.apache.hudi.common.util.ReflectionUtils.isSubClass; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests {@link ReflectionUtils} + */ +public class TestReflectionUtils { + @Test + public void testIsSubClass() { + String subClassName1 = DirectMarkerBasedDetectionStrategy.class.getName(); + String subClassName2 = TimelineServerBasedDetectionStrategy.class.getName(); + assertTrue(isSubClass(subClassName1, EarlyConflictDetectionStrategy.class)); + assertTrue(isSubClass(subClassName2, EarlyConflictDetectionStrategy.class)); + assertTrue(isSubClass(subClassName2, TimelineServerBasedDetectionStrategy.class)); + assertFalse(isSubClass(subClassName2, DirectMarkerBasedDetectionStrategy.class)); + } +} diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java index 2bf6a59b11d8..20604e61c741 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java @@ -188,8 +188,16 @@ public CompletableFuture createMarker(Context context, String markerDir, try { synchronized (earlyConflictDetectionLock) { if (earlyConflictDetectionStrategy == null) { - earlyConflictDetectionStrategy = (TimelineServerBasedDetectionStrategy) ReflectionUtils.loadClass(timelineServiceConfig.earlyConflictDetectionStrategy, - basePath, markerDir, markerName, timelineServiceConfig.checkCommitConflict); + String strategyClassName = timelineServiceConfig.earlyConflictDetectionStrategy; + if (!ReflectionUtils.isSubClass(strategyClassName, TimelineServerBasedDetectionStrategy.class)) { + LOG.warn("Cannot use " + strategyClassName + " for timeline-server-based markers."); + strategyClassName = "org.apache.hudi.timeline.service.handlers.marker.AsyncTimelineServerBasedDetectionStrategy"; + LOG.warn("Falling back to " + strategyClassName); + } + + earlyConflictDetectionStrategy = + (TimelineServerBasedDetectionStrategy) ReflectionUtils.loadClass( + strategyClassName, basePath, markerDir, markerName, timelineServiceConfig.checkCommitConflict); } // markerDir => $base_path/.hoodie/.temp/$instant_time