From 08307d45b4acf598b58b1787e832a2300848db3b Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Mon, 7 Apr 2025 20:32:27 -0700 Subject: [PATCH 01/25] Add initial changes for storage-based lock provider abstract implementation --- .../lock/ConditionalWriteLockConfig.java | 113 ++++ .../lock/ConditionalWriteLockProvider.java | 551 ++++++++++++++++++ .../lock/ConditionalWriteLockConfigTest.java | 110 ++++ .../TestConditionalWriteLockProvider.java | 546 +++++++++++++++++ .../common/table/HoodieTableMetaClient.java | 2 + .../hudi/common/fs/TestStorageSchemes.java | 4 + .../apache/hudi/storage/StorageSchemes.java | 89 +-- 7 files changed, 1380 insertions(+), 35 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java new file mode 100644 index 0000000000000..17e31f250f3de --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + package org.apache.hudi.client.transaction.lock; + + import org.apache.hudi.common.config.ConfigProperty; + import org.apache.hudi.common.config.HoodieConfig; + import org.apache.hudi.common.config.LockConfiguration; + import org.apache.hudi.common.config.TypedProperties; + + import org.apache.hadoop.fs.Path; + + import java.util.concurrent.TimeUnit; + + import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; + import static org.apache.hudi.client.transaction.lock.ConditionalWriteLockProvider.DEFAULT_TABLE_LOCK_FILE_NAME; + import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; + + public class ConditionalWriteLockConfig extends HoodieConfig { + private static final String SINCE_VERSION_0_15_0 = "0.15.0"; + private static final String CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX + "conditional_write."; + public static final ConfigProperty LOCK_INTERNAL_STORAGE_LOCATION = + ConfigProperty.key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "locks_location") + .defaultValue("") + .markAdvanced() + .sinceVersion(SINCE_VERSION_0_15_0) + .withDocumentation( + "For conditional write based lock provider, the optional URI where lock files are written. " + + "Must be the same filesystem as the table path and should conditional writes. " + + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); + + public static final ConfigProperty LOCK_VALIDITY_TIMEOUT_MS = + ConfigProperty.key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "lock_validity_timeout_ms") + .defaultValue(TimeUnit.MINUTES.toMillis(5)) + .markAdvanced() + .sinceVersion(SINCE_VERSION_0_15_0) + .withDocumentation("For storage based conditional write lock provider, the amount of time each new lock is valid for." + + "The lock provider will attempt to renew its lock until it successful extends the lock lease period" + + "or the validity timeout is reached."); + + public static final ConfigProperty HEARTBEAT_POLL_MS = + ConfigProperty.key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "heartbeat_poll_ms") + .defaultValue(TimeUnit.SECONDS.toMillis(30)) + .markAdvanced() + .sinceVersion(SINCE_VERSION_0_15_0) + .withDocumentation("For storage based conditional write lock provider, the amount of time to wait before renewing the lock. Defaults to 30 seconds."); + + public long getLockValidityTimeoutMs() { + return getLong(LOCK_VALIDITY_TIMEOUT_MS); + } + + public long getHeartbeatPollMs() { + return getLong(HEARTBEAT_POLL_MS); + } + + public String getHudiTableBasePath() { + // Required!! + return getString(BASE_PATH.key()); + } + + public String getLocksLocation() { + return getString(LOCK_INTERNAL_STORAGE_LOCATION); + } + + public static class Builder { + private final ConditionalWriteLockConfig lockConfig = new ConditionalWriteLockConfig(); + + public ConditionalWriteLockConfig build() { + lockConfig.setDefaults(ConditionalWriteLockConfig.class.getName()); + return lockConfig; + } + + public ConditionalWriteLockConfig.Builder fromProperties(TypedProperties props) { + lockConfig.getProps().putAll(props); + checkRequiredProps(); + return this; + } + + private void checkRequiredProps() { + String notExistsMsg = " does not exist!"; + if (Boolean.FALSE.equals(lockConfig.contains(BASE_PATH.key()))) { + throw new IllegalArgumentException(BASE_PATH.key() + notExistsMsg); + } + if (lockConfig.getStringOrDefault(LOCK_INTERNAL_STORAGE_LOCATION).startsWith(lockConfig.getHudiTableBasePath())) { + throw new IllegalArgumentException(LOCK_INTERNAL_STORAGE_LOCATION.key() + " cannot start with the hudi table base path."); + } + if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) * 3) { + throw new IllegalArgumentException(LOCK_VALIDITY_TIMEOUT_MS.key() + " should be more than triple " + HEARTBEAT_POLL_MS.key()); + } + if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < 5000) { + throw new IllegalArgumentException(LOCK_VALIDITY_TIMEOUT_MS.key() + " should be greater than or equal to 5 seconds."); + } + if (lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) < 1000) { + throw new IllegalArgumentException(HEARTBEAT_POLL_MS.key() + " should be greater than or equal to 1 second."); + } + } + } + } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java new file mode 100644 index 0000000000000..c867ed8a03d27 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java @@ -0,0 +1,551 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + package org.apache.hudi.client.transaction.lock; + + import org.apache.hudi.client.transaction.lock.models.StorageLockData; + import org.apache.hudi.client.transaction.lock.models.StorageLockFile; + import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; +import org.apache.hudi.client.transaction.lock.models.LockGetResult; +import org.apache.hudi.client.transaction.lock.models.LockProviderHeartbeatManager; +import org.apache.hudi.client.transaction.lock.models.LockUpdateResult; +import org.apache.hudi.common.config.LockConfiguration; + import org.apache.hudi.common.lock.LockProvider; + import org.apache.hudi.common.lock.LockState; + import org.apache.hudi.common.util.CollectionUtils; + import org.apache.hudi.common.util.ReflectionUtils; + import org.apache.hudi.common.util.StringUtils; + import org.apache.hudi.common.util.VisibleForTesting; + import org.apache.hudi.common.util.collection.Pair; + import org.apache.hudi.common.util.hash.HashID; + import org.apache.hudi.exception.HoodieLockException; + import org.apache.hudi.exception.HoodieNotSupportedException; + import org.apache.hudi.storage.StorageConfiguration; + import org.apache.hudi.storage.StoragePath; + import org.apache.hudi.storage.StorageSchemes; + + import org.jetbrains.annotations.NotNull; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + + import javax.annotation.concurrent.GuardedBy; + import javax.annotation.concurrent.ThreadSafe; + + import java.net.URI; + import java.net.URISyntaxException; + import java.util.Map; + import java.util.Objects; + import java.util.Properties; + import java.util.UUID; + import java.util.concurrent.TimeUnit; + + import static org.apache.hudi.common.lock.LockState.ACQUIRED; + import static org.apache.hudi.common.lock.LockState.ACQUIRING; + import static org.apache.hudi.common.lock.LockState.FAILED_TO_ACQUIRE; + import static org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE; + import static org.apache.hudi.common.lock.LockState.RELEASED; + import static org.apache.hudi.common.lock.LockState.RELEASING; + import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; + + @ThreadSafe + public class ConditionalWriteLockProvider implements LockProvider { + + public static final String DEFAULT_TABLE_LOCK_FILE_NAME = "table_lock"; + // How long to wait before retrying lock acquisition in blocking calls. + private static final long DEFAULT_LOCK_ACQUISITION_BUFFER = 1000; + // Maximum expected clock drift between two nodes. + // This is similar idea as SkewAdjustingTimeGenerator. + // In reality, within a single cloud provider all nodes share the same ntp server + // therefore we do not expect drift more than a few ms. + // However, since our lock leases are pretty long, we can use a high buffer. + private static final long CLOCK_DRIFT_BUFFER_MS = 500; + + // When we retry lock upserts, do so 5 times + private static final long LOCK_UPSERT_RETRY_COUNT = 5; + + private static final String GCS_LOCK_SERVICE_CLASS_NAME = "org.apache.hudi.gcp.transaction.lock.GCSStorageLock"; + private static final String S3_LOCK_SERVICE_CLASS_NAME = "org.apache.hudi.aws.transaction.lock.S3StorageLock"; + + // The static logger to be shared across contexts + private static final Logger DEFAULT_LOGGER = LoggerFactory.getLogger(ConditionalWriteLockProvider.class); + private static final String LOCK_STATE_LOGGER_MSG = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}"; + private static final String LOCK_STATE_LOGGER_MSG_WITH_INFO = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}, {}"; + + private static final Map SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS = CollectionUtils.createImmutableMap( + Pair.of(StorageSchemes.GCS.getScheme(), GCS_LOCK_SERVICE_CLASS_NAME), + Pair.of(StorageSchemes.S3A.getScheme(), S3_LOCK_SERVICE_CLASS_NAME), + Pair.of(StorageSchemes.S3.getScheme(), S3_LOCK_SERVICE_CLASS_NAME)); + + @VisibleForTesting + Logger logger; + + // The lock service implementation which interacts with cloud storage. + private final StorageLock lockService; + + // Local variables + private final long heartbeatIntervalMs; + private final long lockValidityMs; + private final String ownerId; + private final String lockFilePath; + private final String bucketName; + private final HeartbeatManager heartbeatManager; + + // Provide a place to store the "current lock object". + @GuardedBy("this") + private StorageLockFile currentLockObj = null; + // Ensures we do not try to lock after being closed. + @GuardedBy("this") + private boolean isClosed = false; + + private synchronized void setLock(StorageLockFile lockObj) { + if (lockObj != null && !Objects.equals(lockObj.getOwner(), this.ownerId)) { + throw new HoodieLockException("Owners do not match! Current LP owner: " + this.ownerId + " lock path: " + this.lockFilePath + " owner: " + lockObj.getOwner()); + } + this.currentLockObj = lockObj; + } + + /** + * Default constructor for ConditionalWriteLockProvider, required by LockManager to instantiate it using reflection. + * @param lockConfiguration The lock configuration, should be transformable into ConditionalWriteLockConfig + * @param conf Storage config, ignored. + */ + public ConditionalWriteLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration conf) { + ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder().fromProperties(lockConfiguration.getConfig()).build(); + heartbeatIntervalMs = config.getHeartbeatPollMs(); + lockValidityMs = config.getLockValidityTimeoutMs(); + + // Determine if the provided locks location is relative. + String configuredLocksLocation = config.getLocksLocation(); + + // If using base path, recalculate the locks location as .hoodie/.locks + String locksLocation = StringUtils.isNullOrEmpty(configuredLocksLocation) + ? String.format("%s%s%s", config.getHudiTableBasePath(), StoragePath.SEPARATOR, LOCKS_FOLDER_NAME) + : configuredLocksLocation; + + URI uri = parseURI(locksLocation); + bucketName = uri.getHost(); // For most schemes, the bucket/container is the host. + String folderName = uri.getPath(); // Path after the bucket/container. + + String fileName = StringUtils.isNullOrEmpty(configuredLocksLocation) + ? DEFAULT_TABLE_LOCK_FILE_NAME + : slugifyLockFolderFromBasePath(config.getHudiTableBasePath()); + + lockFilePath = buildLockObjectPath(folderName, fileName); + ownerId = UUID.randomUUID().toString(); + this.logger = DEFAULT_LOGGER; + this.heartbeatManager = new LockProviderHeartbeatManager( + ownerId, + heartbeatIntervalMs, + this::renewLock + ); + + try { + this.lockService = (StorageLock) ReflectionUtils.loadClass( + getLockServiceClassName(uri.getScheme()), + new Class[] {String.class, String.class, String.class, Properties.class }, + new Object[] {ownerId, bucketName, lockFilePath, lockConfiguration.getConfig()}); + } catch (Throwable e) { + throw new HoodieLockException("Failed to load and initialize StorageLock", e); + } + + logger.info("Instantiated new Conditional Write LP, owner: {}, lockfilePath: {}", ownerId, lockFilePath); + } + + private URI parseURI(String location) { + try { + return new URI(location); + } catch (URISyntaxException e) { + throw new HoodieLockException("Unable to parse locks location as a URI: " + location, e); + } + } + + private static @NotNull String getLockServiceClassName(String scheme) { + if (SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.containsKey(scheme) && StorageSchemes.isConditionalWriteSupported(scheme)) { + return SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.get(scheme); + } else { + throw new HoodieNotSupportedException("No implementation of StorageLock supports this scheme: " + scheme); + } + } + + @VisibleForTesting + ConditionalWriteLockProvider( + int heartbeatIntervalMs, + int lockValidityMs, + String bucketName, + String lockFilePath, + String ownerId, + HeartbeatManager heartbeatManager, + StorageLock lockService, + Logger logger) { + this.heartbeatIntervalMs = heartbeatIntervalMs; + this.lockValidityMs = lockValidityMs; + this.bucketName = bucketName; + this.lockFilePath = lockFilePath; + this.heartbeatManager = heartbeatManager; + this.lockService = lockService; + this.ownerId = ownerId; + this.logger = logger; + logger.debug("Instantiating new Conditional Write LP, owner: {}", ownerId); + } + + // ----------------------------------------- + // BASE METHODS + // ----------------------------------------- + + @Override + public synchronized StorageLockFile getLock() { + return currentLockObj; + } + + /** + * Attempts to acquire the lock within the given timeout. + */ + @Override + public boolean tryLock(long time, TimeUnit unit) { + long deadlineNanos = System.nanoTime() + unit.toNanos(time); + + while (System.nanoTime() < deadlineNanos) { + try { + if (tryLock()) { + return true; + } + logger.debug( + LOCK_STATE_LOGGER_MSG, + ownerId, + lockFilePath, + Thread.currentThread(), + ACQUIRING); + Thread.sleep(DEFAULT_LOCK_ACQUISITION_BUFFER); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new HoodieLockException(generateLockStateMessage(LockState.FAILED_TO_ACQUIRE), e); + } + } + + return false; + } + + @Override + public synchronized void close() { + try { + this.unlock(); + } catch (Exception e) { + logger.error("Owner {}: Failed to unlock current lock.", ownerId, e); + } + try { + this.lockService.close(); + } catch (Exception e) { + logger.error("Owner {}: Lock service failed to close.", ownerId, e); + } + try { + this.heartbeatManager.close(); + } catch (Exception e) { + logger.error("Owner {}: Heartbeat manager failed to close.", ownerId, e); + } + + this.isClosed = true; + } + + private synchronized boolean isLockStillValid(StorageLockFile lock) { + return !lock.isExpired() && !isCurrentTimeCertainlyOlderThanDistributedTime(lock.getValidUntil()); + } + + /** + * Attempts a single pass to acquire the lock (non-blocking). + * @return true if lock acquired, false otherwise + */ + @Override + public synchronized boolean tryLock() { + assertHeartBeatManagerExists(); + assertUnclosed(); + logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), ACQUIRING); + if (actuallyHoldsLock()) { + // Supports reentrant locks + return true; + } + + if (this.heartbeatManager.hasActiveHeartbeat()) { + logger.error("Detected broken invariant: there is an active heartbeat without a lock being held."); + // Breach of object invariant - we should never have an active heartbeat without holding a lock. + throw new HoodieLockException(generateLockStateMessage(FAILED_TO_ACQUIRE)); + } + + Pair latestLock = this.lockService.readCurrentLockFile(); + if (latestLock.getLeft() == LockGetResult.UNKNOWN_ERROR) { + logger.info( + LOCK_STATE_LOGGER_MSG_WITH_INFO, + ownerId, + lockFilePath, + Thread.currentThread(), + FAILED_TO_ACQUIRE, + "Failed to get the latest lock status"); + // We were not able to determine whether a lock was present. + return false; + } + + if (latestLock.getLeft() == LockGetResult.SUCCESS && isLockStillValid(latestLock.getRight())) { + String msg = String.format("Lock already held by %s", latestLock.getRight().getOwner()); + // Lock held by others. + logger.info( + LOCK_STATE_LOGGER_MSG_WITH_INFO, + ownerId, + lockFilePath, + Thread.currentThread(), + FAILED_TO_ACQUIRE, + msg); + return false; + } + + // Try to acquire the lock + StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId); + Pair lockUpdateStatus = this.lockService.tryCreateOrUpdateLockFile( + newLockData, latestLock.getLeft() == LockGetResult.NOT_EXISTS ? null : latestLock.getRight()); + if (lockUpdateStatus.getLeft() != LockUpdateResult.SUCCESS) { + // failed to acquire the lock, indicates concurrent contention + logger.info( + LOCK_STATE_LOGGER_MSG, + ownerId, + lockFilePath, + Thread.currentThread(), + FAILED_TO_ACQUIRE); + return false; + } + this.setLock(lockUpdateStatus.getRight()); + + // There is a remote chance that + // - after lock is acquired but before heartbeat starts the lock is expired. + // - lock is acquired and heartbeat is up yet it does not run timely before the lock is expired + // It is mitigated by setting the lock validity period to a reasonably long period to survive until heartbeat comes, plus + // set the heartbeat interval relatively small enough. + if (!this.heartbeatManager.startHeartbeatForThread(Thread.currentThread())) { + // Precondition "no active heartbeat" is checked previously, so when startHeartbeatForThread returns false, + // we are confident no heartbeat thread is running. + logger.error( + LOCK_STATE_LOGGER_MSG_WITH_INFO, + ownerId, + lockFilePath, + Thread.currentThread(), + RELEASING, + "We were unable to start the heartbeat!"); + tryExpireCurrentLock(); + return false; + } + + logger.info( + LOCK_STATE_LOGGER_MSG, + ownerId, + lockFilePath, + Thread.currentThread(), + ACQUIRED); + return true; + } + + /** + * Determines whether this provider currently holds a valid lock. + * + *

This method checks both the existence of a lock object and its validity. A lock is considered + * valid only if it exists and has not expired according to its timestamp. + * + * @return {@code true} if this provider holds a valid lock, {@code false} otherwise + */ + private boolean actuallyHoldsLock() { + return believesLockMightBeHeld() && isLockStillValid(getLock()); + } + + /** + * Checks if this provider has a non-null lock object reference. + * + *

A non-null lock object indicates that this provider has previously **successfully** acquired a lock via + * ConditionalWriteLockProvider##lock and has not yet **successfully** released it via ConditionalWriteLockProvider#unlock(). + * It is merely an indicator that the lock might be held by this provider. To truly certify we are the owner of the lock, + * ConditionalWriteLockProvider#actuallyHoldsLock should be used. + * + * @return {@code true} if this provider has a non-null lock object, {@code false} otherwise + * @see ConditionalWriteLockProvider#actuallyHoldsLock() + */ + private boolean believesLockMightBeHeld() { + return this.getLock() != null; + } + + /** + * Unlock by marking our current lock file "expired": true. + */ + @Override + public synchronized void unlock() { + assertHeartBeatManagerExists(); + if (!believesLockMightBeHeld()) { + return; + } + boolean believesNoLongerHoldsLock = true; + + // Try to stop the heartbeat first + if (heartbeatManager.hasActiveHeartbeat()) { + logger.debug("Owner {}: Gracefully shutting down heartbeat.", ownerId); + believesNoLongerHoldsLock &= heartbeatManager.stopHeartbeat(true); + } + + // Then expire the current lock. + believesNoLongerHoldsLock &= tryExpireCurrentLock(); + if (!believesNoLongerHoldsLock) { + throw new HoodieLockException(generateLockStateMessage(FAILED_TO_RELEASE)); + } + } + + private void assertHeartBeatManagerExists() { + if (heartbeatManager == null) { + // broken function precondition. + throw new HoodieLockException("Unexpected null heartbeatManager"); + } + } + + private void assertUnclosed() { + if (this.isClosed) { + throw new HoodieLockException("Lock provider already closed"); + } + } + + /** + * Tries to expire the currently held lock. + * @return True if we were successfully able to upload an expired lock. + */ + private synchronized boolean tryExpireCurrentLock() { + // It does not make sense to have heartbeat alive extending the lock lease while here we are trying + // to expire the lock. + if (heartbeatManager.hasActiveHeartbeat()) { + // broken function precondition. + throw new HoodieLockException("Must stop heartbeat before expire lock file"); + } + logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASING); + // Upload metadata that will unlock this lock. + Pair result = this.lockService.tryCreateOrUpdateLockFileWithRetry( + () -> new StorageLockData(true, this.getLock().getValidUntil(), ownerId), + this.getLock(), + // Keep retrying for the normal validity time. + LOCK_UPSERT_RETRY_COUNT); + switch (result.getLeft()) { + case UNKNOWN_ERROR: + // Here we do not know the state of the lock. + logger.error(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), FAILED_TO_RELEASE); + return false; + case SUCCESS: + logger.info(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); + setLock(null); + return true; + case ACQUIRED_BY_OTHERS: + // As we are confident no lock is held by itself, clean up the cached lock object. + logger.warn(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); + setLock(null); + return true; + default: + throw new HoodieLockException("Unexpected lock update result: " + result.getLeft()); + } + } + + /** + * Renews (heartbeats) the current lock if we are the holder, it forcefully set the expiration flag + * to false and the lock expiration time to a later time in the future. + */ + @VisibleForTesting + protected synchronized boolean renewLock() { + try { + // If we don't hold the lock, no-op. + if (!believesLockMightBeHeld()) { + logger.warn("Owner {}: Cannot renew, no lock held by this process", ownerId); + // No need to extend lock lease. + return false; + } + + long oldExpirationMs = getLock().getValidUntil(); + // Attempt conditional update, extend lock. There are 3 cases: + // 1. Happy case: lock has not expired yet, we extend the lease to a longer period. + // 2. Corner case 1: lock is expired and is acquired by others, lock renewal failed with ACQUIRED_BY_OTHERS. + // 3. Corner case 2: lock is expired but no one has acquired it yet, lock renewal "revived" the expired lock. + // Please note we expect the corner cases almost never happens. + // Action taken for corner case 2 is just a best effort mitigation. At least it prevents further data corruption by + // letting someone else acquire the lock. + Pair currentLock = this.lockService.tryCreateOrUpdateLockFileWithRetry( + () -> new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId), + getLock(), + LOCK_UPSERT_RETRY_COUNT); + switch (currentLock.getLeft()) { + case ACQUIRED_BY_OTHERS: + logger.error("Owner {}: Unable to renew lock as it is acquired by others.", ownerId); + // No need to extend lock lease anymore. + return false; + case UNKNOWN_ERROR: + // This could be transient, but unclear, we will let the heartbeat continue normally. + // If the next heartbeat run identifies our lock has expired we will error out. + logger.warn("Owner {}: Unable to renew lock due to unknown error, could be transient.", ownerId); + // Let heartbeat retry later. + return true; + case SUCCESS: + // Only positive outcome + this.setLock(currentLock.getRight()); + logger.info("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}.", + ownerId, oldExpirationMs - System.currentTimeMillis(), lockFilePath); + // Let heartbeat continue to renew lock lease again later. + return true; + default: + throw new HoodieLockException("Unexpected lock update result: " + currentLock.getLeft()); + } + } catch (Exception e) { + logger.error("Owner {}: Exception occurred while renewing lock", ownerId, e); + return false; + } + } + + // --------- + // Utilities + // --------- + + /** + * Method to calculate whether a timestamp from a distributed source has definitively occurred yet. + */ + protected boolean isCurrentTimeCertainlyOlderThanDistributedTime(long epochMs) { + return System.currentTimeMillis() > epochMs + CLOCK_DRIFT_BUFFER_MS; + } + + private String buildLockObjectPath(String lockFolderName, String lockTableFileName) { + // Normalize inputs by removing trailing slashes + // We know lockTableFileName has already been parsed. + if (lockFolderName.startsWith("/")) { + lockFolderName = lockFolderName.substring(1); + } + + // Append a slash only if one isn't already present. + return lockFolderName + (lockFolderName.endsWith("/") ? "" : "/") + lockTableFileName + ".json"; + } + + private String slugifyLockFolderFromBasePath(String basePathKey) { + // Remove the prefix once + String cleanedPath = basePathKey.replaceFirst("^(gs://|s3://|s3a://)", ""); + + // Generate the lock name + return cleanedPath + .replaceAll("[/\\\\]+", "-") // Replace slashes with dashes + .replaceAll("[^0-9a-zA-Z_-]", "-") // Replace invalid characters + .toLowerCase() + .substring(Math.max(0, cleanedPath.length() - 40)) // Get last 40 characters + + "_" + HashID.generateXXHashAsString(basePathKey, HashID.Size.BITS_64); + } + + private String generateLockStateMessage(LockState state) { + String threadName = Thread.currentThread().getName(); + return String.format("Owner %s: Lock file path %s, Thread %s, Conditional Write lock state %s", ownerId, lockFilePath, threadName, state.toString()); + } + } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java new file mode 100644 index 0000000000000..d310bab5e1e34 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + package org.apache.hudi.client.transaction.lock; + + import org.apache.hudi.common.config.TypedProperties; + + import org.junit.jupiter.api.Test; + + import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; + import static org.junit.jupiter.api.Assertions.assertEquals; + import static org.junit.jupiter.api.Assertions.assertThrows; + import static org.junit.jupiter.api.Assertions.assertTrue; + + class ConditionalWriteLockConfigTest { + + @Test + void testDefaultValues() { + TypedProperties props = new TypedProperties(); + props.setProperty(BASE_PATH.key(), "s3://hudi-bucket/table/basepath"); + + ConditionalWriteLockConfig.Builder builder = new ConditionalWriteLockConfig.Builder(); + ConditionalWriteLockConfig config = builder + .fromProperties(props) + .build(); + + assertEquals("", config.getLocksLocation()); + assertEquals(5 * 60 * 1000, config.getLockValidityTimeoutMs(), "Default lock validity should be 5 minutes"); + assertEquals(30 * 1000, config.getHeartbeatPollMs(), "Default heartbeat poll time should be 30 seconds"); + } + + @Test + void testCustomValues() { + TypedProperties props = new TypedProperties(); + props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "10000"); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + + ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() + .fromProperties(props) + .build(); + + assertEquals("s3://bucket/path/locks", config.getLocksLocation()); + assertEquals(120000, config.getLockValidityTimeoutMs()); + assertEquals(10000, config.getHeartbeatPollMs()); + assertEquals("/hudi/table/basepath", config.getHudiTableBasePath()); + } + + @Test + void testMissingRequiredProperties() { + TypedProperties props = new TypedProperties(); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); + ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(BASE_PATH.key())); + props.setProperty(BASE_PATH.key(), "s3://bucket/path/locks"); + props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); + assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + } + + @Test + void testTimeThresholds() { + TypedProperties props = new TypedProperties(); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "3000"); + ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "4999"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); + exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "999"); + exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key())); + } + + @Test + void testBucketPathValidation() { + TypedProperties props = new TypedProperties(); + props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "invalid/path"); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + + ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() + .fromProperties(props) + .build(); + + assertEquals("invalid/path", config.getLocksLocation(), "Locks location should not modify invalid inputs"); + } + } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java new file mode 100644 index 0000000000000..fba1e992b2f0a --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java @@ -0,0 +1,546 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + package org.apache.hudi.client.transaction.lock; + + import org.apache.hudi.client.transaction.lock.models.StorageLockData; + import org.apache.hudi.client.transaction.lock.models.StorageLockFile; + import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; +import org.apache.hudi.client.transaction.lock.models.LockGetResult; +import org.apache.hudi.client.transaction.lock.models.LockUpdateResult; +import org.apache.hudi.common.config.LockConfiguration; + import org.apache.hudi.common.config.TypedProperties; + import org.apache.hudi.common.testutils.HoodieTestUtils; + import org.apache.hudi.common.util.collection.Pair; + import org.apache.hudi.exception.HoodieLockException; + import org.apache.hudi.storage.StorageConfiguration; + + import org.junit.jupiter.api.AfterEach; + import org.junit.jupiter.api.BeforeEach; + import org.junit.jupiter.api.Test; + import org.junit.jupiter.params.ParameterizedTest; + import org.junit.jupiter.params.provider.ValueSource; + import org.slf4j.Logger; + + import java.net.URISyntaxException; + import java.util.UUID; + import java.util.concurrent.CountDownLatch; + import java.util.concurrent.TimeUnit; + import java.util.concurrent.atomic.AtomicInteger; + import java.util.function.Supplier; + + import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; + import static org.junit.jupiter.api.Assertions.*; + import static org.mockito.ArgumentMatchers.any; + import static org.mockito.ArgumentMatchers.anyLong; + import static org.mockito.ArgumentMatchers.eq; + import static org.mockito.ArgumentMatchers.isNull; + import static org.mockito.Mockito.atLeastOnce; + import static org.mockito.Mockito.doAnswer; + import static org.mockito.Mockito.doReturn; + import static org.mockito.Mockito.doThrow; + import static org.mockito.Mockito.mock; + import static org.mockito.Mockito.spy; + import static org.mockito.Mockito.verify; + import static org.mockito.Mockito.when; + + /** + * Unit test class for ConditionalWriteLockProvider + */ + class TestConditionalWriteLockProvider { + private ConditionalWriteLockProvider lockProvider; + private StorageLock mockLockService; + private HeartbeatManager mockHeartbeatManager; + private Logger mockLogger; + private final String ownerId = UUID.randomUUID().toString(); + private static final int DEFAULT_LOCK_VALIDITY_MS = 5000; + + @BeforeEach + void setupLockProvider() { + mockLockService = mock(StorageLock.class); + mockHeartbeatManager = mock(HeartbeatManager.class); + mockLogger = mock(Logger.class); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); + lockProvider = spy(new ConditionalWriteLockProvider( + 1000, + DEFAULT_LOCK_VALIDITY_MS, + "my-bucket", + "gs://bucket/lake/db/tbl-default", + ownerId, + mockHeartbeatManager, + mockLockService, + mockLogger + )); + } + + @AfterEach + void cleanupLockProvider() { + lockProvider.close(); + } + + @Test + void testUnsupportedLockStorageLocation() { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), "hdfs://bucket/lake/db/tbl-default"); + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertTrue(ex.getCause().getMessage().contains("No implementation of StorageLock supports this scheme")); + } + + @Test + void testValidLockStorageLocation() { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); + props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/locks"); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); + } + + @Test + void testInvalidLockStorageLocation() { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); + props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/lake/db/tbl-default/.hoodie/.metadata"); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + assertThrows(IllegalArgumentException.class, () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + } + + @ParameterizedTest + @ValueSource(strings = {"gs://bucket/lake/db/tbl-default", "s3://bucket/lake/db/tbl-default", "s3a://bucket/lake/db/tbl-default"}) + void testNonExistentWriteServiceWithDefaults(String tableBasePathString) { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), tableBasePathString); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); + } + + @Test + void testInvalidLocksLocationForWriteService() { + TypedProperties props = new TypedProperties(); + props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "not a uri"); + props.put(BASE_PATH.key(), "gs://bucket/lake/db/tbl-default"); + props.put(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); + props.put(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + Throwable cause = ex.getCause(); + assertNotNull(cause); + assertInstanceOf(URISyntaxException.class, cause); + assertTrue(ex.getMessage().contains("Unable to parse locks location as a URI")); + } + + @Test + void testTryLockForTimeUnitThrowsOnInterrupt() throws Exception { + doReturn(false).when(lockProvider).tryLock(); + CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread(() -> { + try { + lockProvider.tryLock(1, TimeUnit.SECONDS); + } catch (HoodieLockException e) { + latch.countDown(); + } + }); + t.start(); + Thread.sleep(50); + t.interrupt(); + assertTrue(latch.await(2, TimeUnit.SECONDS)); + } + + @Test + void testTryLockForTimeUnitAcquiresLockEventually() throws Exception { + AtomicInteger count = new AtomicInteger(0); + doAnswer(inv -> count.incrementAndGet() > 2).when(lockProvider).tryLock(); + CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread(() -> { + assertTrue(lockProvider.tryLock(4, TimeUnit.SECONDS)); + latch.countDown(); + }); + t.start(); + assertTrue(latch.await(5, TimeUnit.SECONDS)); + } + + @Test + void testTryLockForTimeUnitFailsToAcquireLockEventually() throws Exception { + AtomicInteger count = new AtomicInteger(0); + doAnswer(inv -> count.incrementAndGet() > 2).when(lockProvider).tryLock(); + CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread(() -> { + assertFalse(lockProvider.tryLock(1, TimeUnit.SECONDS)); + latch.countDown(); + }); + t.start(); + assertTrue(latch.await(2, TimeUnit.SECONDS)); + } + + @Test + void testTryLockSuccess() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + assertEquals(realLockFile, lockProvider.getLock()); + verify(mockLockService, atLeastOnce()).tryCreateOrUpdateLockFile(any(), any()); + } + + @Test + void testTryLockSuccessButFailureToStartHeartbeat() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(false); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(realLockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + + boolean acquired = lockProvider.tryLock(); + assertFalse(acquired); + } + + @Test + void testTryLockFailsFromOwnerMismatch() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockFile returnedLockFile = new StorageLockFile(new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, "different-owner"), "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, returnedLockFile)); + + HoodieLockException ex = assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); + assertTrue(ex.getMessage().contains("Owners do not match!")); + } + + @Test + void testTryLockFailsDueToExistingLock() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, "other-owner"); + StorageLockFile existingLock = new StorageLockFile(data, "v2"); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.SUCCESS, existingLock)); + + boolean acquired = lockProvider.tryLock(); + assertFalse(acquired); + } + + @Test + void testTryLockFailsToUpdateFile() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.ACQUIRED_BY_OTHERS, null)); + assertFalse(lockProvider.tryLock()); + } + + @Test + void testTryLockFailsDueToUnknownState() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.UNKNOWN_ERROR, null)); + assertFalse(lockProvider.tryLock()); + } + + @Test + void testTryLockSucceedsWhenExistingLockExpiredByTime() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, "other-owner"); + StorageLockFile existingLock = new StorageLockFile(data, "v2"); + StorageLockData newData = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(newData, "v1"); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.SUCCESS, existingLock)); + when(mockLockService.tryCreateOrUpdateLockFile(any(), eq(existingLock))).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + } + + @Test + void testTryLockReentrancySucceeds() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + // Re-entrancy succeeds + assertTrue(lockProvider.tryLock()); + } + + @Test + void testTryLockReentrancyAfterLockExpiredByTime() { + // In an extremely unlikely scenario, we could have a local reference to a lock which is present but expired, + // and because we were unable to stop the heartbeat properly, we did not successfully set it to null. + // Due to the nature of the heartbeat manager, this is expected to introduce some delay, but not be permanently blocking. + // There are a few variations of this edge case, so we must test them all. + + // Here the lock is still "unexpired" but the time shows expired. + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile expiredLock = new StorageLockFile(data, "v1"); + doReturn(expiredLock).when(lockProvider).getLock(); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData validData = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile validLock = new StorageLockFile(validData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, validLock)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + assertTrue(lockProvider.tryLock()); + } + + @Test + void testTryLockReentrancyAfterLockSetExpired() { + // In an extremely unlikely scenario, we could have a local reference to a lock which is present but expired, + // and because we were unable to stop the heartbeat properly, we did not successfully set it to null. + // Due to the nature of the heartbeat manager, this is expected to introduce some delay, but not be permanently blocking. + // There are a few variations of this edge case, so we must test them all. + + // Here the lock is "expired" but the time shows unexpired. + StorageLockData data = new StorageLockData(true, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile expiredLock = new StorageLockFile(data, "v1"); + doReturn(expiredLock).when(lockProvider).getLock(); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData validData = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile validLock = new StorageLockFile(validData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, validLock)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + assertTrue(lockProvider.tryLock()); + } + + @Test + void testTryLockHeartbeatStillActive() { + // In an extremely unlikely scenario, we could have a local reference to a lock which is present but expired, + // and because we were unable to stop the heartbeat properly, we did not successfully set it to null. + // Due to the nature of the heartbeat manager, this is expected to introduce some delay, but not be permanently blocking. + // There are a few variations of this edge case, so we must test them all. + + // Here the heartbeat is still active, so we have to error out. + StorageLockData data = new StorageLockData(true, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile expiredLock = new StorageLockFile(data, "v1"); + doReturn(expiredLock).when(lockProvider).getLock(); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); + } + + @Test + void testUnlockSucceedsAndReentrancy() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(realLockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, new StorageLockFile(new StorageLockData(true, data.getValidUntil(), ownerId), "v2"))); + assertTrue(lockProvider.tryLock()); + when(mockHeartbeatManager.hasActiveHeartbeat()) + .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active, return true. + .thenReturn(false); // when try to set lock to expire we will assert no active heartbeat as a precondition. + lockProvider.unlock(); + assertNull(lockProvider.getLock()); + lockProvider.unlock(); + } + + @Test + void testUnlockFailsToStopHeartbeat() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + assertTrue(lockProvider.tryLock()); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(false); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + assertThrows(HoodieLockException.class, () -> lockProvider.unlock()); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + } + + @Test + void testCloseFailsToStopHeartbeat() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + assertTrue(lockProvider.tryLock()); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(false); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + // Should wrap the exception and log error. + lockProvider.close(); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + } + + @Test + void testRenewLockReturnsFalseWhenNoLockHeld() { + doReturn(null).when(lockProvider).getLock(); + assertFalse(lockProvider.renewLock()); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + verify(mockLogger).warn("Owner {}: Cannot renew, no lock held by this process", this.ownerId); + } + + @Test + void testRenewLockWithoutHoldingLock() { + doReturn(null).when(lockProvider).getLock(); + assertFalse(lockProvider.renewLock()); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + verify(mockLogger).warn("Owner {}: Cannot renew, no lock held by this process", this.ownerId); + } + + @Test + void testRenewLockWithFullyExpiredLock() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile nearExpiredLockFile = new StorageLockFile(data, "v1"); + doReturn(nearExpiredLockFile).when(lockProvider).getLock(); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(nearExpiredLockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.ACQUIRED_BY_OTHERS, null)); + assertFalse(lockProvider.renewLock()); + verify(mockLogger).error("Owner {}: Unable to renew lock as it is acquired by others.", this.ownerId); + } + + @Test + void testRenewLockUnableToUpsertLockFileButNotFatal() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + // Signal the upsert attempt failed, but may be transient. See interface for more details. + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.UNKNOWN_ERROR, null)); + assertTrue(lockProvider.renewLock()); + } + + @Test + void testRenewLockUnableToUpsertLockFileFatal() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + // Signal the upsert attempt failed, but may be transient. See interface for more details. + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.UNKNOWN_ERROR, null)); + // renewLock return true so it will be retried. + assertTrue(lockProvider.renewLock()); + + verify(mockLogger).warn("Owner {}: Unable to renew lock due to unknown error, could be transient.", this.ownerId); + } + + @Test + void testRenewLockSucceedsButRenewalWithinExpirationWindow() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + + StorageLockData nearExpirationData = new StorageLockData(false, System.currentTimeMillis(), ownerId); + StorageLockFile lockFileNearExpiration = new StorageLockFile(nearExpirationData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, lockFileNearExpiration)); + + // We used to fail in this case before, but since we are only modifying a single lock file, this is ok now. + // Therefore, this can be a happy path variation. + assertTrue(lockProvider.renewLock()); + } + + @Test + void testRenewLockSucceeds() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + + StorageLockData successData = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile successLockFile = new StorageLockFile(successData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, successLockFile)); + assertTrue(lockProvider.renewLock()); + + verify(mockLogger).info(eq("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}."), eq(this.ownerId), anyLong(), eq("gs://bucket/lake/db/tbl-default")); + } + + @Test + void testRenewLockFails() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenThrow(new RuntimeException("Failure")); + assertFalse(lockProvider.renewLock()); + + verify(mockLogger).error(eq("Owner {}: Exception occurred while renewing lock"), eq(ownerId), any(RuntimeException.class)); + } + + @Test + void testCloseCallsDependencies() throws Exception { + lockProvider.close(); + verify(mockLockService, atLeastOnce()).close(); + verify(mockHeartbeatManager, atLeastOnce()).close(); + assertNull(lockProvider.getLock()); + } + + @Test + void testCloseWithErrorForLockService() throws Exception { + doThrow(new RuntimeException("Some failure")).when(mockLockService).close(); + lockProvider.close(); + verify(mockLogger).error(eq("Owner {}: Lock service failed to close."), eq(ownerId), any(RuntimeException.class)); + assertNull(lockProvider.getLock()); + } + + @Test + void testCloseWithErrorForHeartbeatManager() throws Exception { + doThrow(new RuntimeException("Some failure")).when(mockHeartbeatManager).close(); + lockProvider.close(); + verify(mockLogger).error(eq("Owner {}: Heartbeat manager failed to close."), eq(ownerId), any(RuntimeException.class)); + assertNull(lockProvider.getLock()); + } + + public static class StubStorageLock implements StorageLock { + public StubStorageLock(String arg1, String arg2, String arg3) { + // No-op constructor for reflection + } + + @Override + public Pair tryCreateOrUpdateLockFile(StorageLockData newLockData, StorageLockFile previousLockFile) { + return null; + } + + @Override + public Pair tryCreateOrUpdateLockFileWithRetry( + Supplier newLockDataSupplier, + StorageLockFile previousLockFile, + long retryExpiration) { + return null; + } + + @Override + public Pair readCurrentLockFile() { + return null; + } + + @Override + public void close() throws Exception { + // stub, no-op + } + } + + } + + \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 4c0e947d73b17..c42024f42ebad 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -131,6 +131,8 @@ public class HoodieTableMetaClient implements Serializable { public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + StoragePath.SEPARATOR + ".fileids"; + public static final String LOCKS_FOLDER_NAME = METAFOLDER_NAME + StoragePath.SEPARATOR + ".locks"; + public static final String SCHEMA_FOLDER_NAME = ".schema"; public static final String MARKER_EXTN = ".marker"; diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index e718bc21ed6b8..0245d612f4b57 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -39,6 +39,10 @@ public void testStorageSchemes() { assertTrue(StorageSchemes.isSchemeSupported("afs")); assertFalse(StorageSchemes.isSchemeSupported("s2")); + assertTrue(StorageSchemes.isConditionalWriteSupported("s3")); + assertTrue(StorageSchemes.isConditionalWriteSupported("s3a")); + assertTrue(StorageSchemes.isConditionalWriteSupported("gs")); + assertTrue(StorageSchemes.isAtomicCreationSupported("file")); assertTrue(StorageSchemes.isAtomicCreationSupported("hdfs")); assertFalse(StorageSchemes.isAtomicCreationSupported("afs")); diff --git a/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java b/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java index ca795917f98f8..62f9aa146b321 100644 --- a/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java +++ b/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java @@ -28,65 +28,66 @@ */ public enum StorageSchemes { // Local filesystem - FILE("file", false, true), + FILE("file", false, true, false), // Hadoop File System - HDFS("hdfs", false, true), + HDFS("hdfs", false, true, false), // Baidu Advanced File System - AFS("afs", null, null), + AFS("afs", null, null, false), // Mapr File System - MAPRFS("maprfs", null, null), + MAPRFS("maprfs", null, null, false), // Apache Ignite FS - IGNITE("igfs", null, null), + IGNITE("igfs", null, null, false), // AWS S3 - S3A("s3a", true, null), - S3("s3", true, null), + S3A("s3a", true, null, true), + S3("s3", true, null, true), // Google Cloud Storage - GCS("gs", true, null), + GCS("gs", true, null, true), // Azure WASB - WASB("wasb", null, null), - WASBS("wasbs", null, null), + WASB("wasb", null, null, false), + WASBS("wasbs", null, null, false), // Azure ADLS - ADL("adl", null, null), + ADL("adl", null, null, false), // Azure ADLS Gen2 - ABFS("abfs", null, null), - ABFSS("abfss", null, null), + ABFS("abfs", null, null, false), + ABFSS("abfss", null, null, false), // Aliyun OSS - OSS("oss", null, null), - // View FS for federated setups. If federating across cloud stores, then append support is false + OSS("oss", null, null, false), + // View FS for federated setups. If federating across cloud stores, then append + // support is false // View FS support atomic creation - VIEWFS("viewfs", null, true), - //ALLUXIO - ALLUXIO("alluxio", null, null), + VIEWFS("viewfs", null, true, false), + // ALLUXIO + ALLUXIO("alluxio", null, null, false), // Tencent Cloud Object Storage - COSN("cosn", null, null), + COSN("cosn", null, null, false), // Tencent Cloud HDFS - CHDFS("ofs", null, null), + CHDFS("ofs", null, null, false), // Tencent Cloud CacheFileSystem - GOOSEFS("gfs", null, null), + GOOSEFS("gfs", null, null, false), // Databricks file system - DBFS("dbfs", null, null), + DBFS("dbfs", null, null, false), // IBM Cloud Object Storage - COS("cos", null, null), + COS("cos", null, null, false), // Huawei Cloud Object Storage - OBS("obs", null, null), + OBS("obs", null, null, false), // Kingsoft Standard Storage ks3 - KS3("ks3", null, null), + KS3("ks3", null, null, false), // Netease Object Storage nos - NOS("nos", null, null), + NOS("nos", null, null, false), // JuiceFileSystem - JFS("jfs", null, null), + JFS("jfs", null, null, false), // Baidu Object Storage - BOS("bos", null, null), + BOS("bos", null, null, false), // Oracle Cloud Infrastructure Object Storage - OCI("oci", null, null), + OCI("oci", null, null, false), // Volcengine Object Storage - TOS("tos", null, null), + TOS("tos", null, null, false), // Volcengine Cloud HDFS - CFS("cfs", null, null), + CFS("cfs", null, null, false), // Aliyun Apsara File Storage for HDFS - DFS("dfs", false, true), + DFS("dfs", false, true, false), // Hopsworks File System - HOPSFS("hopsfs", false, true); + HOPSFS("hopsfs", false, true, false); // list files may bring pressure to storage with centralized meta service like HDFS. // when we want to get only part of files under a directory rather than all files, use getStatus may be more friendly than listStatus. @@ -98,11 +99,17 @@ public enum StorageSchemes { private final Boolean isWriteTransactional; // null for uncertain if dfs support atomic create&delete, please update this for each FS private final Boolean supportAtomicCreation; + private final Boolean supportsConditionalWrite; - StorageSchemes(String scheme, Boolean isWriteTransactional, Boolean supportAtomicCreation) { + StorageSchemes( + String scheme, + Boolean isWriteTransactional, + Boolean supportAtomicCreation, + Boolean supportsConditionalWrite) { this.scheme = scheme; this.isWriteTransactional = isWriteTransactional; this.supportAtomicCreation = supportAtomicCreation; + this.supportsConditionalWrite = supportsConditionalWrite; } public String getScheme() { @@ -117,6 +124,10 @@ public boolean isAtomicCreationSupported() { return supportAtomicCreation != null && supportAtomicCreation; } + public boolean isConditionalWriteSupported() { + return supportsConditionalWrite != null && supportsConditionalWrite; + } + public static boolean isSchemeSupported(String scheme) { return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme)); } @@ -143,4 +154,12 @@ public static boolean isListStatusFriendly(String scheme) { return LIST_STATUS_FRIENDLY_SCHEMES.contains(scheme); } -} + + public static boolean isConditionalWriteSupported(String scheme) { + if (!isSchemeSupported(scheme)) { + throw new IllegalArgumentException("Unsupported scheme :" + scheme); + } + return Arrays.stream(StorageSchemes.values()) + .anyMatch(s -> s.isConditionalWriteSupported() && s.scheme.equals(scheme)); + } +} \ No newline at end of file From d61049d95db9bb0307618d6092405752fc5f05e2 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Mon, 7 Apr 2025 21:07:24 -0700 Subject: [PATCH 02/25] fix formatting --- .../lock/ConditionalWriteLockConfig.java | 200 +-- .../lock/ConditionalWriteLockProvider.java | 1086 ++++++++-------- .../lock/ConditionalWriteLockConfigTest.java | 186 +-- .../TestConditionalWriteLockProvider.java | 1092 +++++++++-------- 4 files changed, 1325 insertions(+), 1239 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java index 17e31f250f3de..eda78add50cc4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -16,98 +16,108 @@ * limitations under the License. */ - package org.apache.hudi.client.transaction.lock; - - import org.apache.hudi.common.config.ConfigProperty; - import org.apache.hudi.common.config.HoodieConfig; - import org.apache.hudi.common.config.LockConfiguration; - import org.apache.hudi.common.config.TypedProperties; - - import org.apache.hadoop.fs.Path; - - import java.util.concurrent.TimeUnit; - - import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; - import static org.apache.hudi.client.transaction.lock.ConditionalWriteLockProvider.DEFAULT_TABLE_LOCK_FILE_NAME; - import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; - - public class ConditionalWriteLockConfig extends HoodieConfig { - private static final String SINCE_VERSION_0_15_0 = "0.15.0"; - private static final String CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX + "conditional_write."; - public static final ConfigProperty LOCK_INTERNAL_STORAGE_LOCATION = - ConfigProperty.key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "locks_location") - .defaultValue("") - .markAdvanced() - .sinceVersion(SINCE_VERSION_0_15_0) - .withDocumentation( - "For conditional write based lock provider, the optional URI where lock files are written. " - + "Must be the same filesystem as the table path and should conditional writes. " - + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); - - public static final ConfigProperty LOCK_VALIDITY_TIMEOUT_MS = - ConfigProperty.key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "lock_validity_timeout_ms") - .defaultValue(TimeUnit.MINUTES.toMillis(5)) - .markAdvanced() - .sinceVersion(SINCE_VERSION_0_15_0) - .withDocumentation("For storage based conditional write lock provider, the amount of time each new lock is valid for." - + "The lock provider will attempt to renew its lock until it successful extends the lock lease period" - + "or the validity timeout is reached."); - - public static final ConfigProperty HEARTBEAT_POLL_MS = - ConfigProperty.key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "heartbeat_poll_ms") - .defaultValue(TimeUnit.SECONDS.toMillis(30)) - .markAdvanced() - .sinceVersion(SINCE_VERSION_0_15_0) - .withDocumentation("For storage based conditional write lock provider, the amount of time to wait before renewing the lock. Defaults to 30 seconds."); - - public long getLockValidityTimeoutMs() { - return getLong(LOCK_VALIDITY_TIMEOUT_MS); - } - - public long getHeartbeatPollMs() { - return getLong(HEARTBEAT_POLL_MS); - } - - public String getHudiTableBasePath() { - // Required!! - return getString(BASE_PATH.key()); - } - - public String getLocksLocation() { - return getString(LOCK_INTERNAL_STORAGE_LOCATION); - } - - public static class Builder { - private final ConditionalWriteLockConfig lockConfig = new ConditionalWriteLockConfig(); - - public ConditionalWriteLockConfig build() { - lockConfig.setDefaults(ConditionalWriteLockConfig.class.getName()); - return lockConfig; - } - - public ConditionalWriteLockConfig.Builder fromProperties(TypedProperties props) { - lockConfig.getProps().putAll(props); - checkRequiredProps(); - return this; - } - - private void checkRequiredProps() { - String notExistsMsg = " does not exist!"; - if (Boolean.FALSE.equals(lockConfig.contains(BASE_PATH.key()))) { - throw new IllegalArgumentException(BASE_PATH.key() + notExistsMsg); - } - if (lockConfig.getStringOrDefault(LOCK_INTERNAL_STORAGE_LOCATION).startsWith(lockConfig.getHudiTableBasePath())) { - throw new IllegalArgumentException(LOCK_INTERNAL_STORAGE_LOCATION.key() + " cannot start with the hudi table base path."); - } - if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) * 3) { - throw new IllegalArgumentException(LOCK_VALIDITY_TIMEOUT_MS.key() + " should be more than triple " + HEARTBEAT_POLL_MS.key()); - } - if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < 5000) { - throw new IllegalArgumentException(LOCK_VALIDITY_TIMEOUT_MS.key() + " should be greater than or equal to 5 seconds."); - } - if (lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) < 1000) { - throw new IllegalArgumentException(HEARTBEAT_POLL_MS.key() + " should be greater than or equal to 1 second."); - } - } - } - } \ No newline at end of file +package org.apache.hudi.client.transaction.lock; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.hadoop.fs.Path; + +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; +import static org.apache.hudi.client.transaction.lock.ConditionalWriteLockProvider.DEFAULT_TABLE_LOCK_FILE_NAME; +import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; + +public class ConditionalWriteLockConfig extends HoodieConfig { + private static final String SINCE_VERSION_1_0_2 = "1.0.2"; + private static final String CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX + + "conditional_write."; + public static final ConfigProperty LOCK_INTERNAL_STORAGE_LOCATION = ConfigProperty + .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "locks_location") + .defaultValue("") + .markAdvanced() + .sinceVersion(SINCE_VERSION_1_0_2) + .withDocumentation( + "For conditional write based lock provider, the optional URI where lock files are written. " + + "Must be the same filesystem as the table path and should conditional writes. " + + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); + + public static final ConfigProperty LOCK_VALIDITY_TIMEOUT_MS = ConfigProperty + .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "lock_validity_timeout_ms") + .defaultValue(TimeUnit.MINUTES.toMillis(5)) + .markAdvanced() + .sinceVersion(SINCE_VERSION_1_0_2) + .withDocumentation( + "For storage based conditional write lock provider, the amount of time each new lock is valid for." + + "The lock provider will attempt to renew its lock until it successful extends the lock lease period" + + "or the validity timeout is reached."); + + public static final ConfigProperty HEARTBEAT_POLL_MS = ConfigProperty + .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "heartbeat_poll_ms") + .defaultValue(TimeUnit.SECONDS.toMillis(30)) + .markAdvanced() + .sinceVersion(SINCE_VERSION_1_0_2) + .withDocumentation( + "For storage based conditional write lock provider, the amount of time to wait before renewing the lock. Defaults to 30 seconds."); + + public long getLockValidityTimeoutMs() { + return getLong(LOCK_VALIDITY_TIMEOUT_MS); + } + + public long getHeartbeatPollMs() { + return getLong(HEARTBEAT_POLL_MS); + } + + public String getHudiTableBasePath() { + // Required!! + return getString(BASE_PATH.key()); + } + + public String getLocksLocation() { + return getString(LOCK_INTERNAL_STORAGE_LOCATION); + } + + public static class Builder { + private final ConditionalWriteLockConfig lockConfig = new ConditionalWriteLockConfig(); + + public ConditionalWriteLockConfig build() { + lockConfig.setDefaults(ConditionalWriteLockConfig.class.getName()); + return lockConfig; + } + + public ConditionalWriteLockConfig.Builder fromProperties(TypedProperties props) { + lockConfig.getProps().putAll(props); + checkRequiredProps(); + return this; + } + + private void checkRequiredProps() { + String notExistsMsg = " does not exist!"; + if (Boolean.FALSE.equals(lockConfig.contains(BASE_PATH.key()))) { + throw new IllegalArgumentException(BASE_PATH.key() + notExistsMsg); + } + if (lockConfig.getStringOrDefault(LOCK_INTERNAL_STORAGE_LOCATION) + .startsWith(lockConfig.getHudiTableBasePath())) { + throw new IllegalArgumentException( + LOCK_INTERNAL_STORAGE_LOCATION.key() + " cannot start with the hudi table base path."); + } + if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) + * 3) { + throw new IllegalArgumentException( + LOCK_VALIDITY_TIMEOUT_MS.key() + " should be more than triple " + HEARTBEAT_POLL_MS.key()); + } + if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < 5000) { + throw new IllegalArgumentException( + LOCK_VALIDITY_TIMEOUT_MS.key() + " should be greater than or equal to 5 seconds."); + } + if (lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) < 1000) { + throw new IllegalArgumentException( + HEARTBEAT_POLL_MS.key() + " should be greater than or equal to 1 second."); + } + } + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java index c867ed8a03d27..c08055b94481a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java @@ -16,536 +16,566 @@ * limitations under the License. */ - package org.apache.hudi.client.transaction.lock; +package org.apache.hudi.client.transaction.lock; - import org.apache.hudi.client.transaction.lock.models.StorageLockData; - import org.apache.hudi.client.transaction.lock.models.StorageLockFile; - import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; +import org.apache.hudi.client.transaction.lock.models.StorageLockData; +import org.apache.hudi.client.transaction.lock.models.StorageLockFile; +import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; import org.apache.hudi.client.transaction.lock.models.LockGetResult; import org.apache.hudi.client.transaction.lock.models.LockProviderHeartbeatManager; import org.apache.hudi.client.transaction.lock.models.LockUpdateResult; import org.apache.hudi.common.config.LockConfiguration; - import org.apache.hudi.common.lock.LockProvider; - import org.apache.hudi.common.lock.LockState; - import org.apache.hudi.common.util.CollectionUtils; - import org.apache.hudi.common.util.ReflectionUtils; - import org.apache.hudi.common.util.StringUtils; - import org.apache.hudi.common.util.VisibleForTesting; - import org.apache.hudi.common.util.collection.Pair; - import org.apache.hudi.common.util.hash.HashID; - import org.apache.hudi.exception.HoodieLockException; - import org.apache.hudi.exception.HoodieNotSupportedException; - import org.apache.hudi.storage.StorageConfiguration; - import org.apache.hudi.storage.StoragePath; - import org.apache.hudi.storage.StorageSchemes; - - import org.jetbrains.annotations.NotNull; - import org.slf4j.Logger; - import org.slf4j.LoggerFactory; - - import javax.annotation.concurrent.GuardedBy; - import javax.annotation.concurrent.ThreadSafe; - - import java.net.URI; - import java.net.URISyntaxException; - import java.util.Map; - import java.util.Objects; - import java.util.Properties; - import java.util.UUID; - import java.util.concurrent.TimeUnit; - - import static org.apache.hudi.common.lock.LockState.ACQUIRED; - import static org.apache.hudi.common.lock.LockState.ACQUIRING; - import static org.apache.hudi.common.lock.LockState.FAILED_TO_ACQUIRE; - import static org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE; - import static org.apache.hudi.common.lock.LockState.RELEASED; - import static org.apache.hudi.common.lock.LockState.RELEASING; - import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; - - @ThreadSafe - public class ConditionalWriteLockProvider implements LockProvider { - - public static final String DEFAULT_TABLE_LOCK_FILE_NAME = "table_lock"; - // How long to wait before retrying lock acquisition in blocking calls. - private static final long DEFAULT_LOCK_ACQUISITION_BUFFER = 1000; - // Maximum expected clock drift between two nodes. - // This is similar idea as SkewAdjustingTimeGenerator. - // In reality, within a single cloud provider all nodes share the same ntp server - // therefore we do not expect drift more than a few ms. - // However, since our lock leases are pretty long, we can use a high buffer. - private static final long CLOCK_DRIFT_BUFFER_MS = 500; - - // When we retry lock upserts, do so 5 times - private static final long LOCK_UPSERT_RETRY_COUNT = 5; - - private static final String GCS_LOCK_SERVICE_CLASS_NAME = "org.apache.hudi.gcp.transaction.lock.GCSStorageLock"; - private static final String S3_LOCK_SERVICE_CLASS_NAME = "org.apache.hudi.aws.transaction.lock.S3StorageLock"; - - // The static logger to be shared across contexts - private static final Logger DEFAULT_LOGGER = LoggerFactory.getLogger(ConditionalWriteLockProvider.class); - private static final String LOCK_STATE_LOGGER_MSG = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}"; - private static final String LOCK_STATE_LOGGER_MSG_WITH_INFO = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}, {}"; - - private static final Map SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS = CollectionUtils.createImmutableMap( - Pair.of(StorageSchemes.GCS.getScheme(), GCS_LOCK_SERVICE_CLASS_NAME), - Pair.of(StorageSchemes.S3A.getScheme(), S3_LOCK_SERVICE_CLASS_NAME), - Pair.of(StorageSchemes.S3.getScheme(), S3_LOCK_SERVICE_CLASS_NAME)); - - @VisibleForTesting - Logger logger; - - // The lock service implementation which interacts with cloud storage. - private final StorageLock lockService; - - // Local variables - private final long heartbeatIntervalMs; - private final long lockValidityMs; - private final String ownerId; - private final String lockFilePath; - private final String bucketName; - private final HeartbeatManager heartbeatManager; - - // Provide a place to store the "current lock object". - @GuardedBy("this") - private StorageLockFile currentLockObj = null; - // Ensures we do not try to lock after being closed. - @GuardedBy("this") - private boolean isClosed = false; - - private synchronized void setLock(StorageLockFile lockObj) { - if (lockObj != null && !Objects.equals(lockObj.getOwner(), this.ownerId)) { - throw new HoodieLockException("Owners do not match! Current LP owner: " + this.ownerId + " lock path: " + this.lockFilePath + " owner: " + lockObj.getOwner()); - } - this.currentLockObj = lockObj; - } - - /** - * Default constructor for ConditionalWriteLockProvider, required by LockManager to instantiate it using reflection. - * @param lockConfiguration The lock configuration, should be transformable into ConditionalWriteLockConfig - * @param conf Storage config, ignored. - */ - public ConditionalWriteLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration conf) { - ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder().fromProperties(lockConfiguration.getConfig()).build(); - heartbeatIntervalMs = config.getHeartbeatPollMs(); - lockValidityMs = config.getLockValidityTimeoutMs(); - - // Determine if the provided locks location is relative. - String configuredLocksLocation = config.getLocksLocation(); - - // If using base path, recalculate the locks location as .hoodie/.locks - String locksLocation = StringUtils.isNullOrEmpty(configuredLocksLocation) - ? String.format("%s%s%s", config.getHudiTableBasePath(), StoragePath.SEPARATOR, LOCKS_FOLDER_NAME) - : configuredLocksLocation; - - URI uri = parseURI(locksLocation); - bucketName = uri.getHost(); // For most schemes, the bucket/container is the host. - String folderName = uri.getPath(); // Path after the bucket/container. - - String fileName = StringUtils.isNullOrEmpty(configuredLocksLocation) - ? DEFAULT_TABLE_LOCK_FILE_NAME - : slugifyLockFolderFromBasePath(config.getHudiTableBasePath()); - - lockFilePath = buildLockObjectPath(folderName, fileName); - ownerId = UUID.randomUUID().toString(); - this.logger = DEFAULT_LOGGER; - this.heartbeatManager = new LockProviderHeartbeatManager( - ownerId, - heartbeatIntervalMs, - this::renewLock - ); - - try { - this.lockService = (StorageLock) ReflectionUtils.loadClass( - getLockServiceClassName(uri.getScheme()), - new Class[] {String.class, String.class, String.class, Properties.class }, - new Object[] {ownerId, bucketName, lockFilePath, lockConfiguration.getConfig()}); - } catch (Throwable e) { - throw new HoodieLockException("Failed to load and initialize StorageLock", e); - } - - logger.info("Instantiated new Conditional Write LP, owner: {}, lockfilePath: {}", ownerId, lockFilePath); - } - - private URI parseURI(String location) { - try { - return new URI(location); - } catch (URISyntaxException e) { - throw new HoodieLockException("Unable to parse locks location as a URI: " + location, e); - } - } - - private static @NotNull String getLockServiceClassName(String scheme) { - if (SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.containsKey(scheme) && StorageSchemes.isConditionalWriteSupported(scheme)) { - return SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.get(scheme); - } else { - throw new HoodieNotSupportedException("No implementation of StorageLock supports this scheme: " + scheme); - } - } - - @VisibleForTesting - ConditionalWriteLockProvider( - int heartbeatIntervalMs, - int lockValidityMs, - String bucketName, - String lockFilePath, - String ownerId, - HeartbeatManager heartbeatManager, - StorageLock lockService, - Logger logger) { - this.heartbeatIntervalMs = heartbeatIntervalMs; - this.lockValidityMs = lockValidityMs; - this.bucketName = bucketName; - this.lockFilePath = lockFilePath; - this.heartbeatManager = heartbeatManager; - this.lockService = lockService; - this.ownerId = ownerId; - this.logger = logger; - logger.debug("Instantiating new Conditional Write LP, owner: {}", ownerId); - } - - // ----------------------------------------- - // BASE METHODS - // ----------------------------------------- - - @Override - public synchronized StorageLockFile getLock() { - return currentLockObj; - } - - /** - * Attempts to acquire the lock within the given timeout. - */ - @Override - public boolean tryLock(long time, TimeUnit unit) { - long deadlineNanos = System.nanoTime() + unit.toNanos(time); - - while (System.nanoTime() < deadlineNanos) { - try { - if (tryLock()) { - return true; - } - logger.debug( - LOCK_STATE_LOGGER_MSG, - ownerId, - lockFilePath, - Thread.currentThread(), - ACQUIRING); - Thread.sleep(DEFAULT_LOCK_ACQUISITION_BUFFER); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new HoodieLockException(generateLockStateMessage(LockState.FAILED_TO_ACQUIRE), e); - } - } - - return false; - } - - @Override - public synchronized void close() { - try { - this.unlock(); - } catch (Exception e) { - logger.error("Owner {}: Failed to unlock current lock.", ownerId, e); - } - try { - this.lockService.close(); - } catch (Exception e) { - logger.error("Owner {}: Lock service failed to close.", ownerId, e); - } - try { - this.heartbeatManager.close(); - } catch (Exception e) { - logger.error("Owner {}: Heartbeat manager failed to close.", ownerId, e); - } - - this.isClosed = true; - } - - private synchronized boolean isLockStillValid(StorageLockFile lock) { - return !lock.isExpired() && !isCurrentTimeCertainlyOlderThanDistributedTime(lock.getValidUntil()); - } - - /** - * Attempts a single pass to acquire the lock (non-blocking). - * @return true if lock acquired, false otherwise - */ - @Override - public synchronized boolean tryLock() { - assertHeartBeatManagerExists(); - assertUnclosed(); - logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), ACQUIRING); - if (actuallyHoldsLock()) { - // Supports reentrant locks - return true; - } - - if (this.heartbeatManager.hasActiveHeartbeat()) { - logger.error("Detected broken invariant: there is an active heartbeat without a lock being held."); - // Breach of object invariant - we should never have an active heartbeat without holding a lock. - throw new HoodieLockException(generateLockStateMessage(FAILED_TO_ACQUIRE)); - } - - Pair latestLock = this.lockService.readCurrentLockFile(); - if (latestLock.getLeft() == LockGetResult.UNKNOWN_ERROR) { - logger.info( - LOCK_STATE_LOGGER_MSG_WITH_INFO, - ownerId, - lockFilePath, - Thread.currentThread(), - FAILED_TO_ACQUIRE, - "Failed to get the latest lock status"); - // We were not able to determine whether a lock was present. - return false; - } - - if (latestLock.getLeft() == LockGetResult.SUCCESS && isLockStillValid(latestLock.getRight())) { - String msg = String.format("Lock already held by %s", latestLock.getRight().getOwner()); - // Lock held by others. - logger.info( - LOCK_STATE_LOGGER_MSG_WITH_INFO, - ownerId, - lockFilePath, - Thread.currentThread(), - FAILED_TO_ACQUIRE, - msg); - return false; - } - - // Try to acquire the lock - StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId); - Pair lockUpdateStatus = this.lockService.tryCreateOrUpdateLockFile( - newLockData, latestLock.getLeft() == LockGetResult.NOT_EXISTS ? null : latestLock.getRight()); - if (lockUpdateStatus.getLeft() != LockUpdateResult.SUCCESS) { - // failed to acquire the lock, indicates concurrent contention - logger.info( - LOCK_STATE_LOGGER_MSG, - ownerId, - lockFilePath, - Thread.currentThread(), - FAILED_TO_ACQUIRE); - return false; - } - this.setLock(lockUpdateStatus.getRight()); - - // There is a remote chance that - // - after lock is acquired but before heartbeat starts the lock is expired. - // - lock is acquired and heartbeat is up yet it does not run timely before the lock is expired - // It is mitigated by setting the lock validity period to a reasonably long period to survive until heartbeat comes, plus - // set the heartbeat interval relatively small enough. - if (!this.heartbeatManager.startHeartbeatForThread(Thread.currentThread())) { - // Precondition "no active heartbeat" is checked previously, so when startHeartbeatForThread returns false, - // we are confident no heartbeat thread is running. - logger.error( - LOCK_STATE_LOGGER_MSG_WITH_INFO, - ownerId, - lockFilePath, - Thread.currentThread(), - RELEASING, - "We were unable to start the heartbeat!"); - tryExpireCurrentLock(); - return false; - } - - logger.info( - LOCK_STATE_LOGGER_MSG, - ownerId, - lockFilePath, - Thread.currentThread(), - ACQUIRED); - return true; - } - - /** - * Determines whether this provider currently holds a valid lock. - * - *

This method checks both the existence of a lock object and its validity. A lock is considered - * valid only if it exists and has not expired according to its timestamp. - * - * @return {@code true} if this provider holds a valid lock, {@code false} otherwise - */ - private boolean actuallyHoldsLock() { - return believesLockMightBeHeld() && isLockStillValid(getLock()); - } - - /** - * Checks if this provider has a non-null lock object reference. - * - *

A non-null lock object indicates that this provider has previously **successfully** acquired a lock via - * ConditionalWriteLockProvider##lock and has not yet **successfully** released it via ConditionalWriteLockProvider#unlock(). - * It is merely an indicator that the lock might be held by this provider. To truly certify we are the owner of the lock, - * ConditionalWriteLockProvider#actuallyHoldsLock should be used. - * - * @return {@code true} if this provider has a non-null lock object, {@code false} otherwise - * @see ConditionalWriteLockProvider#actuallyHoldsLock() - */ - private boolean believesLockMightBeHeld() { - return this.getLock() != null; - } - - /** - * Unlock by marking our current lock file "expired": true. - */ - @Override - public synchronized void unlock() { - assertHeartBeatManagerExists(); - if (!believesLockMightBeHeld()) { - return; - } - boolean believesNoLongerHoldsLock = true; - - // Try to stop the heartbeat first - if (heartbeatManager.hasActiveHeartbeat()) { - logger.debug("Owner {}: Gracefully shutting down heartbeat.", ownerId); - believesNoLongerHoldsLock &= heartbeatManager.stopHeartbeat(true); - } - - // Then expire the current lock. - believesNoLongerHoldsLock &= tryExpireCurrentLock(); - if (!believesNoLongerHoldsLock) { - throw new HoodieLockException(generateLockStateMessage(FAILED_TO_RELEASE)); - } - } - - private void assertHeartBeatManagerExists() { - if (heartbeatManager == null) { - // broken function precondition. - throw new HoodieLockException("Unexpected null heartbeatManager"); - } - } - - private void assertUnclosed() { - if (this.isClosed) { - throw new HoodieLockException("Lock provider already closed"); - } - } - - /** - * Tries to expire the currently held lock. - * @return True if we were successfully able to upload an expired lock. - */ - private synchronized boolean tryExpireCurrentLock() { - // It does not make sense to have heartbeat alive extending the lock lease while here we are trying - // to expire the lock. - if (heartbeatManager.hasActiveHeartbeat()) { - // broken function precondition. - throw new HoodieLockException("Must stop heartbeat before expire lock file"); - } - logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASING); - // Upload metadata that will unlock this lock. - Pair result = this.lockService.tryCreateOrUpdateLockFileWithRetry( - () -> new StorageLockData(true, this.getLock().getValidUntil(), ownerId), - this.getLock(), - // Keep retrying for the normal validity time. - LOCK_UPSERT_RETRY_COUNT); - switch (result.getLeft()) { - case UNKNOWN_ERROR: - // Here we do not know the state of the lock. - logger.error(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), FAILED_TO_RELEASE); - return false; - case SUCCESS: - logger.info(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); - setLock(null); - return true; - case ACQUIRED_BY_OTHERS: - // As we are confident no lock is held by itself, clean up the cached lock object. - logger.warn(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); - setLock(null); - return true; - default: - throw new HoodieLockException("Unexpected lock update result: " + result.getLeft()); - } - } - - /** - * Renews (heartbeats) the current lock if we are the holder, it forcefully set the expiration flag - * to false and the lock expiration time to a later time in the future. - */ - @VisibleForTesting - protected synchronized boolean renewLock() { - try { - // If we don't hold the lock, no-op. - if (!believesLockMightBeHeld()) { - logger.warn("Owner {}: Cannot renew, no lock held by this process", ownerId); - // No need to extend lock lease. - return false; - } - - long oldExpirationMs = getLock().getValidUntil(); - // Attempt conditional update, extend lock. There are 3 cases: - // 1. Happy case: lock has not expired yet, we extend the lease to a longer period. - // 2. Corner case 1: lock is expired and is acquired by others, lock renewal failed with ACQUIRED_BY_OTHERS. - // 3. Corner case 2: lock is expired but no one has acquired it yet, lock renewal "revived" the expired lock. - // Please note we expect the corner cases almost never happens. - // Action taken for corner case 2 is just a best effort mitigation. At least it prevents further data corruption by - // letting someone else acquire the lock. - Pair currentLock = this.lockService.tryCreateOrUpdateLockFileWithRetry( - () -> new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId), - getLock(), - LOCK_UPSERT_RETRY_COUNT); - switch (currentLock.getLeft()) { - case ACQUIRED_BY_OTHERS: - logger.error("Owner {}: Unable to renew lock as it is acquired by others.", ownerId); - // No need to extend lock lease anymore. - return false; - case UNKNOWN_ERROR: - // This could be transient, but unclear, we will let the heartbeat continue normally. - // If the next heartbeat run identifies our lock has expired we will error out. - logger.warn("Owner {}: Unable to renew lock due to unknown error, could be transient.", ownerId); - // Let heartbeat retry later. - return true; - case SUCCESS: - // Only positive outcome - this.setLock(currentLock.getRight()); - logger.info("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}.", - ownerId, oldExpirationMs - System.currentTimeMillis(), lockFilePath); - // Let heartbeat continue to renew lock lease again later. - return true; - default: - throw new HoodieLockException("Unexpected lock update result: " + currentLock.getLeft()); - } - } catch (Exception e) { - logger.error("Owner {}: Exception occurred while renewing lock", ownerId, e); - return false; - } - } - - // --------- - // Utilities - // --------- - - /** - * Method to calculate whether a timestamp from a distributed source has definitively occurred yet. - */ - protected boolean isCurrentTimeCertainlyOlderThanDistributedTime(long epochMs) { - return System.currentTimeMillis() > epochMs + CLOCK_DRIFT_BUFFER_MS; - } - - private String buildLockObjectPath(String lockFolderName, String lockTableFileName) { - // Normalize inputs by removing trailing slashes - // We know lockTableFileName has already been parsed. - if (lockFolderName.startsWith("/")) { - lockFolderName = lockFolderName.substring(1); - } - - // Append a slash only if one isn't already present. - return lockFolderName + (lockFolderName.endsWith("/") ? "" : "/") + lockTableFileName + ".json"; - } - - private String slugifyLockFolderFromBasePath(String basePathKey) { - // Remove the prefix once - String cleanedPath = basePathKey.replaceFirst("^(gs://|s3://|s3a://)", ""); - - // Generate the lock name - return cleanedPath - .replaceAll("[/\\\\]+", "-") // Replace slashes with dashes - .replaceAll("[^0-9a-zA-Z_-]", "-") // Replace invalid characters - .toLowerCase() - .substring(Math.max(0, cleanedPath.length() - 40)) // Get last 40 characters - + "_" + HashID.generateXXHashAsString(basePathKey, HashID.Size.BITS_64); - } - - private String generateLockStateMessage(LockState state) { - String threadName = Thread.currentThread().getName(); - return String.format("Owner %s: Lock file path %s, Thread %s, Conditional Write lock state %s", ownerId, lockFilePath, threadName, state.toString()); - } - } \ No newline at end of file +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.lock.LockState; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.util.hash.HashID; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StorageSchemes; + +import org.jetbrains.annotations.NotNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.lock.LockState.ACQUIRED; +import static org.apache.hudi.common.lock.LockState.ACQUIRING; +import static org.apache.hudi.common.lock.LockState.FAILED_TO_ACQUIRE; +import static org.apache.hudi.common.lock.LockState.FAILED_TO_RELEASE; +import static org.apache.hudi.common.lock.LockState.RELEASED; +import static org.apache.hudi.common.lock.LockState.RELEASING; +import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; + +@ThreadSafe +public class ConditionalWriteLockProvider implements LockProvider { + + public static final String DEFAULT_TABLE_LOCK_FILE_NAME = "table_lock"; + // How long to wait before retrying lock acquisition in blocking calls. + private static final long DEFAULT_LOCK_ACQUISITION_BUFFER = 1000; + // Maximum expected clock drift between two nodes. + // This is similar idea as SkewAdjustingTimeGenerator. + // In reality, within a single cloud provider all nodes share the same ntp + // server + // therefore we do not expect drift more than a few ms. + // However, since our lock leases are pretty long, we can use a high buffer. + private static final long CLOCK_DRIFT_BUFFER_MS = 500; + + // When we retry lock upserts, do so 5 times + private static final long LOCK_UPSERT_RETRY_COUNT = 5; + + private static final String GCS_LOCK_SERVICE_CLASS_NAME = "org.apache.hudi.gcp.transaction.lock.GCSStorageLock"; + private static final String S3_LOCK_SERVICE_CLASS_NAME = "org.apache.hudi.aws.transaction.lock.S3StorageLock"; + + // The static logger to be shared across contexts + private static final Logger DEFAULT_LOGGER = LoggerFactory.getLogger(ConditionalWriteLockProvider.class); + private static final String LOCK_STATE_LOGGER_MSG = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}"; + private static final String LOCK_STATE_LOGGER_MSG_WITH_INFO = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}, {}"; + + private static final Map SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS = CollectionUtils.createImmutableMap( + Pair.of(StorageSchemes.GCS.getScheme(), GCS_LOCK_SERVICE_CLASS_NAME), + Pair.of(StorageSchemes.S3A.getScheme(), S3_LOCK_SERVICE_CLASS_NAME), + Pair.of(StorageSchemes.S3.getScheme(), S3_LOCK_SERVICE_CLASS_NAME)); + + @VisibleForTesting + Logger logger; + + // The lock service implementation which interacts with cloud storage. + private final StorageLock lockService; + + // Local variables + private final long heartbeatIntervalMs; + private final long lockValidityMs; + private final String ownerId; + private final String lockFilePath; + private final String bucketName; + private final HeartbeatManager heartbeatManager; + + // Provide a place to store the "current lock object". + @GuardedBy("this") + private StorageLockFile currentLockObj = null; + // Ensures we do not try to lock after being closed. + @GuardedBy("this") + private boolean isClosed = false; + + private synchronized void setLock(StorageLockFile lockObj) { + if (lockObj != null && !Objects.equals(lockObj.getOwner(), this.ownerId)) { + throw new HoodieLockException("Owners do not match! Current LP owner: " + this.ownerId + " lock path: " + + this.lockFilePath + " owner: " + lockObj.getOwner()); + } + this.currentLockObj = lockObj; + } + + /** + * Default constructor for ConditionalWriteLockProvider, required by LockManager + * to instantiate it using reflection. + * + * @param lockConfiguration The lock configuration, should be transformable into + * ConditionalWriteLockConfig + * @param conf Storage config, ignored. + */ + public ConditionalWriteLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration conf) { + ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() + .fromProperties(lockConfiguration.getConfig()).build(); + heartbeatIntervalMs = config.getHeartbeatPollMs(); + lockValidityMs = config.getLockValidityTimeoutMs(); + + // Determine if the provided locks location is relative. + String configuredLocksLocation = config.getLocksLocation(); + + // If using base path, recalculate the locks location as .hoodie/.locks + String locksLocation = StringUtils.isNullOrEmpty(configuredLocksLocation) + ? String.format("%s%s%s", config.getHudiTableBasePath(), StoragePath.SEPARATOR, LOCKS_FOLDER_NAME) + : configuredLocksLocation; + + URI uri = parseURI(locksLocation); + bucketName = uri.getHost(); // For most schemes, the bucket/container is the host. + String folderName = uri.getPath(); // Path after the bucket/container. + + String fileName = StringUtils.isNullOrEmpty(configuredLocksLocation) + ? DEFAULT_TABLE_LOCK_FILE_NAME + : slugifyLockFolderFromBasePath(config.getHudiTableBasePath()); + + lockFilePath = buildLockObjectPath(folderName, fileName); + ownerId = UUID.randomUUID().toString(); + this.logger = DEFAULT_LOGGER; + this.heartbeatManager = new LockProviderHeartbeatManager( + ownerId, + heartbeatIntervalMs, + this::renewLock); + + try { + this.lockService = (StorageLock) ReflectionUtils.loadClass( + getLockServiceClassName(uri.getScheme()), + new Class[] { String.class, String.class, String.class, Properties.class }, + new Object[] { ownerId, bucketName, lockFilePath, lockConfiguration.getConfig() }); + } catch (Throwable e) { + throw new HoodieLockException("Failed to load and initialize StorageLock", e); + } + + logger.info("Instantiated new Conditional Write LP, owner: {}, lockfilePath: {}", ownerId, lockFilePath); + } + + private URI parseURI(String location) { + try { + return new URI(location); + } catch (URISyntaxException e) { + throw new HoodieLockException("Unable to parse locks location as a URI: " + location, e); + } + } + + private static @NotNull String getLockServiceClassName(String scheme) { + if (SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.containsKey(scheme) + && StorageSchemes.isConditionalWriteSupported(scheme)) { + return SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.get(scheme); + } else { + throw new HoodieNotSupportedException("No implementation of StorageLock supports this scheme: " + scheme); + } + } + + @VisibleForTesting + ConditionalWriteLockProvider( + int heartbeatIntervalMs, + int lockValidityMs, + String bucketName, + String lockFilePath, + String ownerId, + HeartbeatManager heartbeatManager, + StorageLock lockService, + Logger logger) { + this.heartbeatIntervalMs = heartbeatIntervalMs; + this.lockValidityMs = lockValidityMs; + this.bucketName = bucketName; + this.lockFilePath = lockFilePath; + this.heartbeatManager = heartbeatManager; + this.lockService = lockService; + this.ownerId = ownerId; + this.logger = logger; + logger.debug("Instantiating new Conditional Write LP, owner: {}", ownerId); + } + + // ----------------------------------------- + // BASE METHODS + // ----------------------------------------- + + @Override + public synchronized StorageLockFile getLock() { + return currentLockObj; + } + + /** + * Attempts to acquire the lock within the given timeout. + */ + @Override + public boolean tryLock(long time, TimeUnit unit) { + long deadlineNanos = System.nanoTime() + unit.toNanos(time); + + while (System.nanoTime() < deadlineNanos) { + try { + if (tryLock()) { + return true; + } + logger.debug( + LOCK_STATE_LOGGER_MSG, + ownerId, + lockFilePath, + Thread.currentThread(), + ACQUIRING); + Thread.sleep(DEFAULT_LOCK_ACQUISITION_BUFFER); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new HoodieLockException(generateLockStateMessage(LockState.FAILED_TO_ACQUIRE), e); + } + } + + return false; + } + + @Override + public synchronized void close() { + try { + this.unlock(); + } catch (Exception e) { + logger.error("Owner {}: Failed to unlock current lock.", ownerId, e); + } + try { + this.lockService.close(); + } catch (Exception e) { + logger.error("Owner {}: Lock service failed to close.", ownerId, e); + } + try { + this.heartbeatManager.close(); + } catch (Exception e) { + logger.error("Owner {}: Heartbeat manager failed to close.", ownerId, e); + } + + this.isClosed = true; + } + + private synchronized boolean isLockStillValid(StorageLockFile lock) { + return !lock.isExpired() && !isCurrentTimeCertainlyOlderThanDistributedTime(lock.getValidUntil()); + } + + /** + * Attempts a single pass to acquire the lock (non-blocking). + * + * @return true if lock acquired, false otherwise + */ + @Override + public synchronized boolean tryLock() { + assertHeartBeatManagerExists(); + assertUnclosed(); + logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), ACQUIRING); + if (actuallyHoldsLock()) { + // Supports reentrant locks + return true; + } + + if (this.heartbeatManager.hasActiveHeartbeat()) { + logger.error("Detected broken invariant: there is an active heartbeat without a lock being held."); + // Breach of object invariant - we should never have an active heartbeat without + // holding a lock. + throw new HoodieLockException(generateLockStateMessage(FAILED_TO_ACQUIRE)); + } + + Pair latestLock = this.lockService.readCurrentLockFile(); + if (latestLock.getLeft() == LockGetResult.UNKNOWN_ERROR) { + logger.info( + LOCK_STATE_LOGGER_MSG_WITH_INFO, + ownerId, + lockFilePath, + Thread.currentThread(), + FAILED_TO_ACQUIRE, + "Failed to get the latest lock status"); + // We were not able to determine whether a lock was present. + return false; + } + + if (latestLock.getLeft() == LockGetResult.SUCCESS && isLockStillValid(latestLock.getRight())) { + String msg = String.format("Lock already held by %s", latestLock.getRight().getOwner()); + // Lock held by others. + logger.info( + LOCK_STATE_LOGGER_MSG_WITH_INFO, + ownerId, + lockFilePath, + Thread.currentThread(), + FAILED_TO_ACQUIRE, + msg); + return false; + } + + // Try to acquire the lock + StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId); + Pair lockUpdateStatus = this.lockService.tryCreateOrUpdateLockFile( + newLockData, latestLock.getLeft() == LockGetResult.NOT_EXISTS ? null : latestLock.getRight()); + if (lockUpdateStatus.getLeft() != LockUpdateResult.SUCCESS) { + // failed to acquire the lock, indicates concurrent contention + logger.info( + LOCK_STATE_LOGGER_MSG, + ownerId, + lockFilePath, + Thread.currentThread(), + FAILED_TO_ACQUIRE); + return false; + } + this.setLock(lockUpdateStatus.getRight()); + + // There is a remote chance that + // - after lock is acquired but before heartbeat starts the lock is expired. + // - lock is acquired and heartbeat is up yet it does not run timely before the + // lock is expired + // It is mitigated by setting the lock validity period to a reasonably long + // period to survive until heartbeat comes, plus + // set the heartbeat interval relatively small enough. + if (!this.heartbeatManager.startHeartbeatForThread(Thread.currentThread())) { + // Precondition "no active heartbeat" is checked previously, so when + // startHeartbeatForThread returns false, + // we are confident no heartbeat thread is running. + logger.error( + LOCK_STATE_LOGGER_MSG_WITH_INFO, + ownerId, + lockFilePath, + Thread.currentThread(), + RELEASING, + "We were unable to start the heartbeat!"); + tryExpireCurrentLock(); + return false; + } + + logger.info( + LOCK_STATE_LOGGER_MSG, + ownerId, + lockFilePath, + Thread.currentThread(), + ACQUIRED); + return true; + } + + /** + * Determines whether this provider currently holds a valid lock. + * + *

+ * This method checks both the existence of a lock object and its validity. A + * lock is considered + * valid only if it exists and has not expired according to its timestamp. + * + * @return {@code true} if this provider holds a valid lock, {@code false} + * otherwise + */ + private boolean actuallyHoldsLock() { + return believesLockMightBeHeld() && isLockStillValid(getLock()); + } + + /** + * Checks if this provider has a non-null lock object reference. + * + *

+ * A non-null lock object indicates that this provider has previously + * **successfully** acquired a lock via + * ConditionalWriteLockProvider##lock and has not yet **successfully** released + * it via ConditionalWriteLockProvider#unlock(). + * It is merely an indicator that the lock might be held by this provider. To + * truly certify we are the owner of the lock, + * ConditionalWriteLockProvider#actuallyHoldsLock should be used. + * + * @return {@code true} if this provider has a non-null lock object, + * {@code false} otherwise + * @see ConditionalWriteLockProvider#actuallyHoldsLock() + */ + private boolean believesLockMightBeHeld() { + return this.getLock() != null; + } + + /** + * Unlock by marking our current lock file "expired": true. + */ + @Override + public synchronized void unlock() { + assertHeartBeatManagerExists(); + if (!believesLockMightBeHeld()) { + return; + } + boolean believesNoLongerHoldsLock = true; + + // Try to stop the heartbeat first + if (heartbeatManager.hasActiveHeartbeat()) { + logger.debug("Owner {}: Gracefully shutting down heartbeat.", ownerId); + believesNoLongerHoldsLock &= heartbeatManager.stopHeartbeat(true); + } + + // Then expire the current lock. + believesNoLongerHoldsLock &= tryExpireCurrentLock(); + if (!believesNoLongerHoldsLock) { + throw new HoodieLockException(generateLockStateMessage(FAILED_TO_RELEASE)); + } + } + + private void assertHeartBeatManagerExists() { + if (heartbeatManager == null) { + // broken function precondition. + throw new HoodieLockException("Unexpected null heartbeatManager"); + } + } + + private void assertUnclosed() { + if (this.isClosed) { + throw new HoodieLockException("Lock provider already closed"); + } + } + + /** + * Tries to expire the currently held lock. + * + * @return True if we were successfully able to upload an expired lock. + */ + private synchronized boolean tryExpireCurrentLock() { + // It does not make sense to have heartbeat alive extending the lock lease while + // here we are trying + // to expire the lock. + if (heartbeatManager.hasActiveHeartbeat()) { + // broken function precondition. + throw new HoodieLockException("Must stop heartbeat before expire lock file"); + } + logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASING); + // Upload metadata that will unlock this lock. + Pair result = this.lockService.tryCreateOrUpdateLockFileWithRetry( + () -> new StorageLockData(true, this.getLock().getValidUntil(), ownerId), + this.getLock(), + // Keep retrying for the normal validity time. + LOCK_UPSERT_RETRY_COUNT); + switch (result.getLeft()) { + case UNKNOWN_ERROR: + // Here we do not know the state of the lock. + logger.error(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), FAILED_TO_RELEASE); + return false; + case SUCCESS: + logger.info(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); + setLock(null); + return true; + case ACQUIRED_BY_OTHERS: + // As we are confident no lock is held by itself, clean up the cached lock + // object. + logger.warn(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); + setLock(null); + return true; + default: + throw new HoodieLockException("Unexpected lock update result: " + result.getLeft()); + } + } + + /** + * Renews (heartbeats) the current lock if we are the holder, it forcefully set + * the expiration flag + * to false and the lock expiration time to a later time in the future. + */ + @VisibleForTesting + protected synchronized boolean renewLock() { + try { + // If we don't hold the lock, no-op. + if (!believesLockMightBeHeld()) { + logger.warn("Owner {}: Cannot renew, no lock held by this process", ownerId); + // No need to extend lock lease. + return false; + } + + long oldExpirationMs = getLock().getValidUntil(); + // Attempt conditional update, extend lock. There are 3 cases: + // 1. Happy case: lock has not expired yet, we extend the lease to a longer + // period. + // 2. Corner case 1: lock is expired and is acquired by others, lock renewal + // failed with ACQUIRED_BY_OTHERS. + // 3. Corner case 2: lock is expired but no one has acquired it yet, lock + // renewal "revived" the expired lock. + // Please note we expect the corner cases almost never happens. + // Action taken for corner case 2 is just a best effort mitigation. At least it + // prevents further data corruption by + // letting someone else acquire the lock. + Pair currentLock = this.lockService.tryCreateOrUpdateLockFileWithRetry( + () -> new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId), + getLock(), + LOCK_UPSERT_RETRY_COUNT); + switch (currentLock.getLeft()) { + case ACQUIRED_BY_OTHERS: + logger.error("Owner {}: Unable to renew lock as it is acquired by others.", ownerId); + // No need to extend lock lease anymore. + return false; + case UNKNOWN_ERROR: + // This could be transient, but unclear, we will let the heartbeat continue + // normally. + // If the next heartbeat run identifies our lock has expired we will error out. + logger.warn("Owner {}: Unable to renew lock due to unknown error, could be transient.", ownerId); + // Let heartbeat retry later. + return true; + case SUCCESS: + // Only positive outcome + this.setLock(currentLock.getRight()); + logger.info("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}.", + ownerId, oldExpirationMs - System.currentTimeMillis(), lockFilePath); + // Let heartbeat continue to renew lock lease again later. + return true; + default: + throw new HoodieLockException("Unexpected lock update result: " + currentLock.getLeft()); + } + } catch (Exception e) { + logger.error("Owner {}: Exception occurred while renewing lock", ownerId, e); + return false; + } + } + + // --------- + // Utilities + // --------- + + /** + * Method to calculate whether a timestamp from a distributed source has + * definitively occurred yet. + */ + protected boolean isCurrentTimeCertainlyOlderThanDistributedTime(long epochMs) { + return System.currentTimeMillis() > epochMs + CLOCK_DRIFT_BUFFER_MS; + } + + private String buildLockObjectPath(String lockFolderName, String lockTableFileName) { + // Normalize inputs by removing trailing slashes + // We know lockTableFileName has already been parsed. + if (lockFolderName.startsWith("/")) { + lockFolderName = lockFolderName.substring(1); + } + + // Append a slash only if one isn't already present. + return lockFolderName + (lockFolderName.endsWith("/") ? "" : "/") + lockTableFileName + ".json"; + } + + private String slugifyLockFolderFromBasePath(String basePathKey) { + // Remove the prefix once + String cleanedPath = basePathKey.replaceFirst("^(gs://|s3://|s3a://)", ""); + + // Generate the lock name + return cleanedPath + .replaceAll("[/\\\\]+", "-") // Replace slashes with dashes + .replaceAll("[^0-9a-zA-Z_-]", "-") // Replace invalid characters + .toLowerCase() + .substring(Math.max(0, cleanedPath.length() - 40)) // Get last 40 characters + + "_" + HashID.generateXXHashAsString(basePathKey, HashID.Size.BITS_64); + } + + private String generateLockStateMessage(LockState state) { + String threadName = Thread.currentThread().getName(); + return String.format("Owner %s: Lock file path %s, Thread %s, Conditional Write lock state %s", ownerId, + lockFilePath, threadName, state.toString()); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java index d310bab5e1e34..d4cc9b3311071 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java @@ -16,95 +16,97 @@ * limitations under the License. */ - package org.apache.hudi.client.transaction.lock; - - import org.apache.hudi.common.config.TypedProperties; - - import org.junit.jupiter.api.Test; - - import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; - import static org.junit.jupiter.api.Assertions.assertEquals; - import static org.junit.jupiter.api.Assertions.assertThrows; - import static org.junit.jupiter.api.Assertions.assertTrue; - - class ConditionalWriteLockConfigTest { - - @Test - void testDefaultValues() { - TypedProperties props = new TypedProperties(); - props.setProperty(BASE_PATH.key(), "s3://hudi-bucket/table/basepath"); - - ConditionalWriteLockConfig.Builder builder = new ConditionalWriteLockConfig.Builder(); - ConditionalWriteLockConfig config = builder - .fromProperties(props) - .build(); - - assertEquals("", config.getLocksLocation()); - assertEquals(5 * 60 * 1000, config.getLockValidityTimeoutMs(), "Default lock validity should be 5 minutes"); - assertEquals(30 * 1000, config.getHeartbeatPollMs(), "Default heartbeat poll time should be 30 seconds"); - } - - @Test - void testCustomValues() { - TypedProperties props = new TypedProperties(); - props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "10000"); - props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - - ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() - .fromProperties(props) - .build(); - - assertEquals("s3://bucket/path/locks", config.getLocksLocation()); - assertEquals(120000, config.getLockValidityTimeoutMs()); - assertEquals(10000, config.getHeartbeatPollMs()); - assertEquals("/hudi/table/basepath", config.getHudiTableBasePath()); - } - - @Test - void testMissingRequiredProperties() { - TypedProperties props = new TypedProperties(); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); - ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); - - IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(BASE_PATH.key())); - props.setProperty(BASE_PATH.key(), "s3://bucket/path/locks"); - props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); - assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - } - - @Test - void testTimeThresholds() { - TypedProperties props = new TypedProperties(); - props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "3000"); - ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); - - IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "4999"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); - exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "999"); - exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key())); - } - - @Test - void testBucketPathValidation() { - TypedProperties props = new TypedProperties(); - props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "invalid/path"); - props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - - ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() - .fromProperties(props) - .build(); - - assertEquals("invalid/path", config.getLocksLocation(), "Locks location should not modify invalid inputs"); - } - } \ No newline at end of file +package org.apache.hudi.client.transaction.lock; + +import org.apache.hudi.common.config.TypedProperties; + +import org.junit.jupiter.api.Test; + +import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ConditionalWriteLockConfigTest { + + @Test + void testDefaultValues() { + TypedProperties props = new TypedProperties(); + props.setProperty(BASE_PATH.key(), "s3://hudi-bucket/table/basepath"); + + ConditionalWriteLockConfig.Builder builder = new ConditionalWriteLockConfig.Builder(); + ConditionalWriteLockConfig config = builder + .fromProperties(props) + .build(); + + assertEquals("", config.getLocksLocation()); + assertEquals(5 * 60 * 1000, config.getLockValidityTimeoutMs(), "Default lock validity should be 5 minutes"); + assertEquals(30 * 1000, config.getHeartbeatPollMs(), "Default heartbeat poll time should be 30 seconds"); + } + + @Test + void testCustomValues() { + TypedProperties props = new TypedProperties(); + props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "10000"); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + + ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() + .fromProperties(props) + .build(); + + assertEquals("s3://bucket/path/locks", config.getLocksLocation()); + assertEquals(120000, config.getLockValidityTimeoutMs()); + assertEquals(10000, config.getHeartbeatPollMs()); + assertEquals("/hudi/table/basepath", config.getHudiTableBasePath()); + } + + @Test + void testMissingRequiredProperties() { + TypedProperties props = new TypedProperties(); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); + ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, + () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(BASE_PATH.key())); + props.setProperty(BASE_PATH.key(), "s3://bucket/path/locks"); + props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); + assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + } + + @Test + void testTimeThresholds() { + TypedProperties props = new TypedProperties(); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "3000"); + ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, + () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "4999"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); + exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); + props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); + props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "999"); + exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key())); + } + + @Test + void testBucketPathValidation() { + TypedProperties props = new TypedProperties(); + props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "invalid/path"); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + + ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() + .fromProperties(props) + .build(); + + assertEquals("invalid/path", config.getLocksLocation(), "Locks location should not modify invalid inputs"); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java index fba1e992b2f0a..b3dbc0e76e496 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java @@ -16,531 +16,575 @@ * limitations under the License. */ - package org.apache.hudi.client.transaction.lock; +package org.apache.hudi.client.transaction.lock; - import org.apache.hudi.client.transaction.lock.models.StorageLockData; - import org.apache.hudi.client.transaction.lock.models.StorageLockFile; - import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; +import org.apache.hudi.client.transaction.lock.models.StorageLockData; +import org.apache.hudi.client.transaction.lock.models.StorageLockFile; +import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; import org.apache.hudi.client.transaction.lock.models.LockGetResult; import org.apache.hudi.client.transaction.lock.models.LockUpdateResult; import org.apache.hudi.common.config.LockConfiguration; - import org.apache.hudi.common.config.TypedProperties; - import org.apache.hudi.common.testutils.HoodieTestUtils; - import org.apache.hudi.common.util.collection.Pair; - import org.apache.hudi.exception.HoodieLockException; - import org.apache.hudi.storage.StorageConfiguration; - - import org.junit.jupiter.api.AfterEach; - import org.junit.jupiter.api.BeforeEach; - import org.junit.jupiter.api.Test; - import org.junit.jupiter.params.ParameterizedTest; - import org.junit.jupiter.params.provider.ValueSource; - import org.slf4j.Logger; - - import java.net.URISyntaxException; - import java.util.UUID; - import java.util.concurrent.CountDownLatch; - import java.util.concurrent.TimeUnit; - import java.util.concurrent.atomic.AtomicInteger; - import java.util.function.Supplier; - - import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; - import static org.junit.jupiter.api.Assertions.*; - import static org.mockito.ArgumentMatchers.any; - import static org.mockito.ArgumentMatchers.anyLong; - import static org.mockito.ArgumentMatchers.eq; - import static org.mockito.ArgumentMatchers.isNull; - import static org.mockito.Mockito.atLeastOnce; - import static org.mockito.Mockito.doAnswer; - import static org.mockito.Mockito.doReturn; - import static org.mockito.Mockito.doThrow; - import static org.mockito.Mockito.mock; - import static org.mockito.Mockito.spy; - import static org.mockito.Mockito.verify; - import static org.mockito.Mockito.when; - - /** - * Unit test class for ConditionalWriteLockProvider - */ - class TestConditionalWriteLockProvider { - private ConditionalWriteLockProvider lockProvider; - private StorageLock mockLockService; - private HeartbeatManager mockHeartbeatManager; - private Logger mockLogger; - private final String ownerId = UUID.randomUUID().toString(); - private static final int DEFAULT_LOCK_VALIDITY_MS = 5000; - - @BeforeEach - void setupLockProvider() { - mockLockService = mock(StorageLock.class); - mockHeartbeatManager = mock(HeartbeatManager.class); - mockLogger = mock(Logger.class); - when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); - lockProvider = spy(new ConditionalWriteLockProvider( - 1000, - DEFAULT_LOCK_VALIDITY_MS, - "my-bucket", - "gs://bucket/lake/db/tbl-default", - ownerId, - mockHeartbeatManager, - mockLockService, - mockLogger - )); - } - - @AfterEach - void cleanupLockProvider() { - lockProvider.close(); - } - - @Test - void testUnsupportedLockStorageLocation() { - TypedProperties props = new TypedProperties(); - props.put(BASE_PATH.key(), "hdfs://bucket/lake/db/tbl-default"); - LockConfiguration lockConf = new LockConfiguration(props); - StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); - assertTrue(ex.getCause().getMessage().contains("No implementation of StorageLock supports this scheme")); - } - - @Test - void testValidLockStorageLocation() { - TypedProperties props = new TypedProperties(); - props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); - props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/locks"); - - LockConfiguration lockConf = new LockConfiguration(props); - StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - - HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); - assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); - } - - @Test - void testInvalidLockStorageLocation() { - TypedProperties props = new TypedProperties(); - props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); - props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/lake/db/tbl-default/.hoodie/.metadata"); - - LockConfiguration lockConf = new LockConfiguration(props); - StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - - assertThrows(IllegalArgumentException.class, () -> new ConditionalWriteLockProvider(lockConf, storageConf)); - } - - @ParameterizedTest - @ValueSource(strings = {"gs://bucket/lake/db/tbl-default", "s3://bucket/lake/db/tbl-default", "s3a://bucket/lake/db/tbl-default"}) - void testNonExistentWriteServiceWithDefaults(String tableBasePathString) { - TypedProperties props = new TypedProperties(); - props.put(BASE_PATH.key(), tableBasePathString); - - LockConfiguration lockConf = new LockConfiguration(props); - StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - - HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); - assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); - } - - @Test - void testInvalidLocksLocationForWriteService() { - TypedProperties props = new TypedProperties(); - props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "not a uri"); - props.put(BASE_PATH.key(), "gs://bucket/lake/db/tbl-default"); - props.put(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); - props.put(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); - - LockConfiguration lockConf = new LockConfiguration(props); - StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - - HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); - Throwable cause = ex.getCause(); - assertNotNull(cause); - assertInstanceOf(URISyntaxException.class, cause); - assertTrue(ex.getMessage().contains("Unable to parse locks location as a URI")); - } - - @Test - void testTryLockForTimeUnitThrowsOnInterrupt() throws Exception { - doReturn(false).when(lockProvider).tryLock(); - CountDownLatch latch = new CountDownLatch(1); - Thread t = new Thread(() -> { - try { - lockProvider.tryLock(1, TimeUnit.SECONDS); - } catch (HoodieLockException e) { - latch.countDown(); - } - }); - t.start(); - Thread.sleep(50); - t.interrupt(); - assertTrue(latch.await(2, TimeUnit.SECONDS)); - } - - @Test - void testTryLockForTimeUnitAcquiresLockEventually() throws Exception { - AtomicInteger count = new AtomicInteger(0); - doAnswer(inv -> count.incrementAndGet() > 2).when(lockProvider).tryLock(); - CountDownLatch latch = new CountDownLatch(1); - Thread t = new Thread(() -> { - assertTrue(lockProvider.tryLock(4, TimeUnit.SECONDS)); - latch.countDown(); - }); - t.start(); - assertTrue(latch.await(5, TimeUnit.SECONDS)); - } - - @Test - void testTryLockForTimeUnitFailsToAcquireLockEventually() throws Exception { - AtomicInteger count = new AtomicInteger(0); - doAnswer(inv -> count.incrementAndGet() > 2).when(lockProvider).tryLock(); - CountDownLatch latch = new CountDownLatch(1); - Thread t = new Thread(() -> { - assertFalse(lockProvider.tryLock(1, TimeUnit.SECONDS)); - latch.countDown(); - }); - t.start(); - assertTrue(latch.await(2, TimeUnit.SECONDS)); - } - - @Test - void testTryLockSuccess() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(data, "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - - boolean acquired = lockProvider.tryLock(); - assertTrue(acquired); - assertEquals(realLockFile, lockProvider.getLock()); - verify(mockLockService, atLeastOnce()).tryCreateOrUpdateLockFile(any(), any()); - } - - @Test - void testTryLockSuccessButFailureToStartHeartbeat() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(data, "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(false); - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(realLockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - - boolean acquired = lockProvider.tryLock(); - assertFalse(acquired); - } - - @Test - void testTryLockFailsFromOwnerMismatch() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockFile returnedLockFile = new StorageLockFile(new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, "different-owner"), "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, returnedLockFile)); - - HoodieLockException ex = assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); - assertTrue(ex.getMessage().contains("Owners do not match!")); - } - - @Test - void testTryLockFailsDueToExistingLock() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, "other-owner"); - StorageLockFile existingLock = new StorageLockFile(data, "v2"); - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.SUCCESS, existingLock)); - - boolean acquired = lockProvider.tryLock(); - assertFalse(acquired); - } - - @Test - void testTryLockFailsToUpdateFile() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.ACQUIRED_BY_OTHERS, null)); - assertFalse(lockProvider.tryLock()); - } - - @Test - void testTryLockFailsDueToUnknownState() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.UNKNOWN_ERROR, null)); - assertFalse(lockProvider.tryLock()); - } - - @Test - void testTryLockSucceedsWhenExistingLockExpiredByTime() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, "other-owner"); - StorageLockFile existingLock = new StorageLockFile(data, "v2"); - StorageLockData newData = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(newData, "v1"); - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.SUCCESS, existingLock)); - when(mockLockService.tryCreateOrUpdateLockFile(any(), eq(existingLock))).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - boolean acquired = lockProvider.tryLock(); - assertTrue(acquired); - } - - @Test - void testTryLockReentrancySucceeds() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(data, "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - - boolean acquired = lockProvider.tryLock(); - assertTrue(acquired); - // Re-entrancy succeeds - assertTrue(lockProvider.tryLock()); - } - - @Test - void testTryLockReentrancyAfterLockExpiredByTime() { - // In an extremely unlikely scenario, we could have a local reference to a lock which is present but expired, - // and because we were unable to stop the heartbeat properly, we did not successfully set it to null. - // Due to the nature of the heartbeat manager, this is expected to introduce some delay, but not be permanently blocking. - // There are a few variations of this edge case, so we must test them all. - - // Here the lock is still "unexpired" but the time shows expired. - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile expiredLock = new StorageLockFile(data, "v1"); - doReturn(expiredLock).when(lockProvider).getLock(); - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData validData = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile validLock = new StorageLockFile(validData, "v2"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, validLock)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - - assertTrue(lockProvider.tryLock()); - } - - @Test - void testTryLockReentrancyAfterLockSetExpired() { - // In an extremely unlikely scenario, we could have a local reference to a lock which is present but expired, - // and because we were unable to stop the heartbeat properly, we did not successfully set it to null. - // Due to the nature of the heartbeat manager, this is expected to introduce some delay, but not be permanently blocking. - // There are a few variations of this edge case, so we must test them all. - - // Here the lock is "expired" but the time shows unexpired. - StorageLockData data = new StorageLockData(true, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile expiredLock = new StorageLockFile(data, "v1"); - doReturn(expiredLock).when(lockProvider).getLock(); - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData validData = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile validLock = new StorageLockFile(validData, "v2"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, validLock)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - - assertTrue(lockProvider.tryLock()); - } - - @Test - void testTryLockHeartbeatStillActive() { - // In an extremely unlikely scenario, we could have a local reference to a lock which is present but expired, - // and because we were unable to stop the heartbeat properly, we did not successfully set it to null. - // Due to the nature of the heartbeat manager, this is expected to introduce some delay, but not be permanently blocking. - // There are a few variations of this edge case, so we must test them all. - - // Here the heartbeat is still active, so we have to error out. - StorageLockData data = new StorageLockData(true, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile expiredLock = new StorageLockFile(data, "v1"); - doReturn(expiredLock).when(lockProvider).getLock(); - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); - assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); - } - - @Test - void testUnlockSucceedsAndReentrancy() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(data, "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(realLockFile), anyLong())) - .thenReturn(Pair.of(LockUpdateResult.SUCCESS, new StorageLockFile(new StorageLockData(true, data.getValidUntil(), ownerId), "v2"))); - assertTrue(lockProvider.tryLock()); - when(mockHeartbeatManager.hasActiveHeartbeat()) - .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active, return true. - .thenReturn(false); // when try to set lock to expire we will assert no active heartbeat as a precondition. - lockProvider.unlock(); - assertNull(lockProvider.getLock()); - lockProvider.unlock(); - } - - @Test - void testUnlockFailsToStopHeartbeat() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(data, "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - assertTrue(lockProvider.tryLock()); - when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(false); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); - assertThrows(HoodieLockException.class, () -> lockProvider.unlock()); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); - } - - @Test - void testCloseFailsToStopHeartbeat() { - when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile realLockFile = new StorageLockFile(data, "v1"); - when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); - when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); - assertTrue(lockProvider.tryLock()); - when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(false); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); - // Should wrap the exception and log error. - lockProvider.close(); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); - } - - @Test - void testRenewLockReturnsFalseWhenNoLockHeld() { - doReturn(null).when(lockProvider).getLock(); - assertFalse(lockProvider.renewLock()); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); - verify(mockLogger).warn("Owner {}: Cannot renew, no lock held by this process", this.ownerId); - } - - @Test - void testRenewLockWithoutHoldingLock() { - doReturn(null).when(lockProvider).getLock(); - assertFalse(lockProvider.renewLock()); - when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); - verify(mockLogger).warn("Owner {}: Cannot renew, no lock held by this process", this.ownerId); - } - - @Test - void testRenewLockWithFullyExpiredLock() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile nearExpiredLockFile = new StorageLockFile(data, "v1"); - doReturn(nearExpiredLockFile).when(lockProvider).getLock(); - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(nearExpiredLockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.ACQUIRED_BY_OTHERS, null)); - assertFalse(lockProvider.renewLock()); - verify(mockLogger).error("Owner {}: Unable to renew lock as it is acquired by others.", this.ownerId); - } - - @Test - void testRenewLockUnableToUpsertLockFileButNotFatal() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile lockFile = new StorageLockFile(data, "v1"); - doReturn(lockFile).when(lockProvider).getLock(); - // Signal the upsert attempt failed, but may be transient. See interface for more details. - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.UNKNOWN_ERROR, null)); - assertTrue(lockProvider.renewLock()); - } - - @Test - void testRenewLockUnableToUpsertLockFileFatal() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile lockFile = new StorageLockFile(data, "v1"); - doReturn(lockFile).when(lockProvider).getLock(); - // Signal the upsert attempt failed, but may be transient. See interface for more details. - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.UNKNOWN_ERROR, null)); - // renewLock return true so it will be retried. - assertTrue(lockProvider.renewLock()); - - verify(mockLogger).warn("Owner {}: Unable to renew lock due to unknown error, could be transient.", this.ownerId); - } - - @Test - void testRenewLockSucceedsButRenewalWithinExpirationWindow() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile lockFile = new StorageLockFile(data, "v1"); - doReturn(lockFile).when(lockProvider).getLock(); - - StorageLockData nearExpirationData = new StorageLockData(false, System.currentTimeMillis(), ownerId); - StorageLockFile lockFileNearExpiration = new StorageLockFile(nearExpirationData, "v2"); - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, lockFileNearExpiration)); - - // We used to fail in this case before, but since we are only modifying a single lock file, this is ok now. - // Therefore, this can be a happy path variation. - assertTrue(lockProvider.renewLock()); - } - - @Test - void testRenewLockSucceeds() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile lockFile = new StorageLockFile(data, "v1"); - doReturn(lockFile).when(lockProvider).getLock(); - - StorageLockData successData = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile successLockFile = new StorageLockFile(successData, "v2"); - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenReturn(Pair.of(LockUpdateResult.SUCCESS, successLockFile)); - assertTrue(lockProvider.renewLock()); - - verify(mockLogger).info(eq("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}."), eq(this.ownerId), anyLong(), eq("gs://bucket/lake/db/tbl-default")); - } - - @Test - void testRenewLockFails() { - StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); - StorageLockFile lockFile = new StorageLockFile(data, "v1"); - doReturn(lockFile).when(lockProvider).getLock(); - - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())).thenThrow(new RuntimeException("Failure")); - assertFalse(lockProvider.renewLock()); - - verify(mockLogger).error(eq("Owner {}: Exception occurred while renewing lock"), eq(ownerId), any(RuntimeException.class)); - } - - @Test - void testCloseCallsDependencies() throws Exception { - lockProvider.close(); - verify(mockLockService, atLeastOnce()).close(); - verify(mockHeartbeatManager, atLeastOnce()).close(); - assertNull(lockProvider.getLock()); - } - - @Test - void testCloseWithErrorForLockService() throws Exception { - doThrow(new RuntimeException("Some failure")).when(mockLockService).close(); - lockProvider.close(); - verify(mockLogger).error(eq("Owner {}: Lock service failed to close."), eq(ownerId), any(RuntimeException.class)); - assertNull(lockProvider.getLock()); - } - - @Test - void testCloseWithErrorForHeartbeatManager() throws Exception { - doThrow(new RuntimeException("Some failure")).when(mockHeartbeatManager).close(); - lockProvider.close(); - verify(mockLogger).error(eq("Owner {}: Heartbeat manager failed to close."), eq(ownerId), any(RuntimeException.class)); - assertNull(lockProvider.getLock()); - } - - public static class StubStorageLock implements StorageLock { - public StubStorageLock(String arg1, String arg2, String arg3) { - // No-op constructor for reflection - } - - @Override - public Pair tryCreateOrUpdateLockFile(StorageLockData newLockData, StorageLockFile previousLockFile) { - return null; - } - - @Override - public Pair tryCreateOrUpdateLockFileWithRetry( - Supplier newLockDataSupplier, - StorageLockFile previousLockFile, - long retryExpiration) { - return null; - } - - @Override - public Pair readCurrentLockFile() { - return null; - } - - @Override - public void close() throws Exception { - // stub, no-op - } - } - - } - - \ No newline at end of file +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.hudi.storage.StorageConfiguration; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.slf4j.Logger; + +import java.net.URISyntaxException; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit test class for ConditionalWriteLockProvider + */ +class TestConditionalWriteLockProvider { + private ConditionalWriteLockProvider lockProvider; + private StorageLock mockLockService; + private HeartbeatManager mockHeartbeatManager; + private Logger mockLogger; + private final String ownerId = UUID.randomUUID().toString(); + private static final int DEFAULT_LOCK_VALIDITY_MS = 5000; + + @BeforeEach + void setupLockProvider() { + mockLockService = mock(StorageLock.class); + mockHeartbeatManager = mock(HeartbeatManager.class); + mockLogger = mock(Logger.class); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); + lockProvider = spy(new ConditionalWriteLockProvider( + 1000, + DEFAULT_LOCK_VALIDITY_MS, + "my-bucket", + "gs://bucket/lake/db/tbl-default", + ownerId, + mockHeartbeatManager, + mockLockService, + mockLogger)); + } + + @AfterEach + void cleanupLockProvider() { + lockProvider.close(); + } + + @Test + void testUnsupportedLockStorageLocation() { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), "hdfs://bucket/lake/db/tbl-default"); + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertTrue(ex.getCause().getMessage().contains("No implementation of StorageLock supports this scheme")); + } + + @Test + void testValidLockStorageLocation() { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); + props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/locks"); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); + } + + @Test + void testInvalidLockStorageLocation() { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); + props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), + "s3://bucket/lake/db/tbl-default/.hoodie/.metadata"); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + assertThrows(IllegalArgumentException.class, () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + } + + @ParameterizedTest + @ValueSource(strings = { "gs://bucket/lake/db/tbl-default", "s3://bucket/lake/db/tbl-default", + "s3a://bucket/lake/db/tbl-default" }) + void testNonExistentWriteServiceWithDefaults(String tableBasePathString) { + TypedProperties props = new TypedProperties(); + props.put(BASE_PATH.key(), tableBasePathString); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); + } + + @Test + void testInvalidLocksLocationForWriteService() { + TypedProperties props = new TypedProperties(); + props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "not a uri"); + props.put(BASE_PATH.key(), "gs://bucket/lake/db/tbl-default"); + props.put(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); + props.put(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); + + LockConfiguration lockConf = new LockConfiguration(props); + StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); + + HoodieLockException ex = assertThrows(HoodieLockException.class, + () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + Throwable cause = ex.getCause(); + assertNotNull(cause); + assertInstanceOf(URISyntaxException.class, cause); + assertTrue(ex.getMessage().contains("Unable to parse locks location as a URI")); + } + + @Test + void testTryLockForTimeUnitThrowsOnInterrupt() throws Exception { + doReturn(false).when(lockProvider).tryLock(); + CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread(() -> { + try { + lockProvider.tryLock(1, TimeUnit.SECONDS); + } catch (HoodieLockException e) { + latch.countDown(); + } + }); + t.start(); + Thread.sleep(50); + t.interrupt(); + assertTrue(latch.await(2, TimeUnit.SECONDS)); + } + + @Test + void testTryLockForTimeUnitAcquiresLockEventually() throws Exception { + AtomicInteger count = new AtomicInteger(0); + doAnswer(inv -> count.incrementAndGet() > 2).when(lockProvider).tryLock(); + CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread(() -> { + assertTrue(lockProvider.tryLock(4, TimeUnit.SECONDS)); + latch.countDown(); + }); + t.start(); + assertTrue(latch.await(5, TimeUnit.SECONDS)); + } + + @Test + void testTryLockForTimeUnitFailsToAcquireLockEventually() throws Exception { + AtomicInteger count = new AtomicInteger(0); + doAnswer(inv -> count.incrementAndGet() > 2).when(lockProvider).tryLock(); + CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread(() -> { + assertFalse(lockProvider.tryLock(1, TimeUnit.SECONDS)); + latch.countDown(); + }); + t.start(); + assertTrue(latch.await(2, TimeUnit.SECONDS)); + } + + @Test + void testTryLockSuccess() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + assertEquals(realLockFile, lockProvider.getLock()); + verify(mockLockService, atLeastOnce()).tryCreateOrUpdateLockFile(any(), any()); + } + + @Test + void testTryLockSuccessButFailureToStartHeartbeat() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(false); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(realLockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + + boolean acquired = lockProvider.tryLock(); + assertFalse(acquired); + } + + @Test + void testTryLockFailsFromOwnerMismatch() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockFile returnedLockFile = new StorageLockFile( + new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, "different-owner"), "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, returnedLockFile)); + + HoodieLockException ex = assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); + assertTrue(ex.getMessage().contains("Owners do not match!")); + } + + @Test + void testTryLockFailsDueToExistingLock() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, + "other-owner"); + StorageLockFile existingLock = new StorageLockFile(data, "v2"); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.SUCCESS, existingLock)); + + boolean acquired = lockProvider.tryLock(); + assertFalse(acquired); + } + + @Test + void testTryLockFailsToUpdateFile() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.ACQUIRED_BY_OTHERS, null)); + assertFalse(lockProvider.tryLock()); + } + + @Test + void testTryLockFailsDueToUnknownState() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.UNKNOWN_ERROR, null)); + assertFalse(lockProvider.tryLock()); + } + + @Test + void testTryLockSucceedsWhenExistingLockExpiredByTime() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, + "other-owner"); + StorageLockFile existingLock = new StorageLockFile(data, "v2"); + StorageLockData newData = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, + ownerId); + StorageLockFile realLockFile = new StorageLockFile(newData, "v1"); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.SUCCESS, existingLock)); + when(mockLockService.tryCreateOrUpdateLockFile(any(), eq(existingLock))) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + } + + @Test + void testTryLockReentrancySucceeds() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + // Re-entrancy succeeds + assertTrue(lockProvider.tryLock()); + } + + @Test + void testTryLockReentrancyAfterLockExpiredByTime() { + // In an extremely unlikely scenario, we could have a local reference to a lock + // which is present but expired, + // and because we were unable to stop the heartbeat properly, we did not + // successfully set it to null. + // Due to the nature of the heartbeat manager, this is expected to introduce + // some delay, but not be permanently blocking. + // There are a few variations of this edge case, so we must test them all. + + // Here the lock is still "unexpired" but the time shows expired. + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile expiredLock = new StorageLockFile(data, "v1"); + doReturn(expiredLock).when(lockProvider).getLock(); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData validData = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, + ownerId); + StorageLockFile validLock = new StorageLockFile(validData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, validLock)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + assertTrue(lockProvider.tryLock()); + } + + @Test + void testTryLockReentrancyAfterLockSetExpired() { + // In an extremely unlikely scenario, we could have a local reference to a lock + // which is present but expired, + // and because we were unable to stop the heartbeat properly, we did not + // successfully set it to null. + // Due to the nature of the heartbeat manager, this is expected to introduce + // some delay, but not be permanently blocking. + // There are a few variations of this edge case, so we must test them all. + + // Here the lock is "expired" but the time shows unexpired. + StorageLockData data = new StorageLockData(true, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile expiredLock = new StorageLockFile(data, "v1"); + doReturn(expiredLock).when(lockProvider).getLock(); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData validData = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, + ownerId); + StorageLockFile validLock = new StorageLockFile(validData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, validLock)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + assertTrue(lockProvider.tryLock()); + } + + @Test + void testTryLockHeartbeatStillActive() { + // In an extremely unlikely scenario, we could have a local reference to a lock + // which is present but expired, + // and because we were unable to stop the heartbeat properly, we did not + // successfully set it to null. + // Due to the nature of the heartbeat manager, this is expected to introduce + // some delay, but not be permanently blocking. + // There are a few variations of this edge case, so we must test them all. + + // Here the heartbeat is still active, so we have to error out. + StorageLockData data = new StorageLockData(true, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile expiredLock = new StorageLockFile(data, "v1"); + doReturn(expiredLock).when(lockProvider).getLock(); + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); + } + + @Test + void testUnlockSucceedsAndReentrancy() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(realLockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, + new StorageLockFile(new StorageLockData(true, data.getValidUntil(), ownerId), "v2"))); + assertTrue(lockProvider.tryLock()); + when(mockHeartbeatManager.hasActiveHeartbeat()) + .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active, + // return true. + .thenReturn(false); // when try to set lock to expire we will assert no active heartbeat as a + // precondition. + lockProvider.unlock(); + assertNull(lockProvider.getLock()); + lockProvider.unlock(); + } + + @Test + void testUnlockFailsToStopHeartbeat() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + assertTrue(lockProvider.tryLock()); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(false); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + assertThrows(HoodieLockException.class, () -> lockProvider.unlock()); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + } + + @Test + void testCloseFailsToStopHeartbeat() { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + assertTrue(lockProvider.tryLock()); + when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(false); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + // Should wrap the exception and log error. + lockProvider.close(); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + } + + @Test + void testRenewLockReturnsFalseWhenNoLockHeld() { + doReturn(null).when(lockProvider).getLock(); + assertFalse(lockProvider.renewLock()); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(true); + verify(mockLogger).warn("Owner {}: Cannot renew, no lock held by this process", this.ownerId); + } + + @Test + void testRenewLockWithoutHoldingLock() { + doReturn(null).when(lockProvider).getLock(); + assertFalse(lockProvider.renewLock()); + when(mockHeartbeatManager.hasActiveHeartbeat()).thenReturn(false); + verify(mockLogger).warn("Owner {}: Cannot renew, no lock held by this process", this.ownerId); + } + + @Test + void testRenewLockWithFullyExpiredLock() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() - DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile nearExpiredLockFile = new StorageLockFile(data, "v1"); + doReturn(nearExpiredLockFile).when(lockProvider).getLock(); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(nearExpiredLockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.ACQUIRED_BY_OTHERS, null)); + assertFalse(lockProvider.renewLock()); + verify(mockLogger).error("Owner {}: Unable to renew lock as it is acquired by others.", this.ownerId); + } + + @Test + void testRenewLockUnableToUpsertLockFileButNotFatal() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + // Signal the upsert attempt failed, but may be transient. See interface for + // more details. + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.UNKNOWN_ERROR, null)); + assertTrue(lockProvider.renewLock()); + } + + @Test + void testRenewLockUnableToUpsertLockFileFatal() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + // Signal the upsert attempt failed, but may be transient. See interface for + // more details. + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.UNKNOWN_ERROR, null)); + // renewLock return true so it will be retried. + assertTrue(lockProvider.renewLock()); + + verify(mockLogger).warn("Owner {}: Unable to renew lock due to unknown error, could be transient.", this.ownerId); + } + + @Test + void testRenewLockSucceedsButRenewalWithinExpirationWindow() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + + StorageLockData nearExpirationData = new StorageLockData(false, System.currentTimeMillis(), ownerId); + StorageLockFile lockFileNearExpiration = new StorageLockFile(nearExpirationData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, lockFileNearExpiration)); + + // We used to fail in this case before, but since we are only modifying a single + // lock file, this is ok now. + // Therefore, this can be a happy path variation. + assertTrue(lockProvider.renewLock()); + } + + @Test + void testRenewLockSucceeds() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + + StorageLockData successData = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, + ownerId); + StorageLockFile successLockFile = new StorageLockFile(successData, "v2"); + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, successLockFile)); + assertTrue(lockProvider.renewLock()); + + verify(mockLogger).info( + eq("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}."), + eq(this.ownerId), anyLong(), eq("gs://bucket/lake/db/tbl-default")); + } + + @Test + void testRenewLockFails() { + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile lockFile = new StorageLockFile(data, "v1"); + doReturn(lockFile).when(lockProvider).getLock(); + + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(any(), eq(lockFile), anyLong())) + .thenThrow(new RuntimeException("Failure")); + assertFalse(lockProvider.renewLock()); + + verify(mockLogger).error(eq("Owner {}: Exception occurred while renewing lock"), eq(ownerId), + any(RuntimeException.class)); + } + + @Test + void testCloseCallsDependencies() throws Exception { + lockProvider.close(); + verify(mockLockService, atLeastOnce()).close(); + verify(mockHeartbeatManager, atLeastOnce()).close(); + assertNull(lockProvider.getLock()); + } + + @Test + void testCloseWithErrorForLockService() throws Exception { + doThrow(new RuntimeException("Some failure")).when(mockLockService).close(); + lockProvider.close(); + verify(mockLogger).error(eq("Owner {}: Lock service failed to close."), eq(ownerId), any(RuntimeException.class)); + assertNull(lockProvider.getLock()); + } + + @Test + void testCloseWithErrorForHeartbeatManager() throws Exception { + doThrow(new RuntimeException("Some failure")).when(mockHeartbeatManager).close(); + lockProvider.close(); + verify(mockLogger).error(eq("Owner {}: Heartbeat manager failed to close."), eq(ownerId), + any(RuntimeException.class)); + assertNull(lockProvider.getLock()); + } + + public static class StubStorageLock implements StorageLock { + public StubStorageLock(String arg1, String arg2, String arg3) { + // No-op constructor for reflection + } + + @Override + public Pair tryCreateOrUpdateLockFile(StorageLockData newLockData, + StorageLockFile previousLockFile) { + return null; + } + + @Override + public Pair tryCreateOrUpdateLockFileWithRetry( + Supplier newLockDataSupplier, + StorageLockFile previousLockFile, + long retryExpiration) { + return null; + } + + @Override + public Pair readCurrentLockFile() { + return null; + } + + @Override + public void close() throws Exception { + // stub, no-op + } + } + +} From 6a999a7bf9a74d443dbdbce1fadd53b195ff5baa Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Mon, 7 Apr 2025 21:36:30 -0700 Subject: [PATCH 03/25] checkstyle --- .../transaction/lock/TestConditionalWriteLockProvider.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java index b3dbc0e76e496..83a3a1aaf2233 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java @@ -389,10 +389,8 @@ void testUnlockSucceedsAndReentrancy() { new StorageLockFile(new StorageLockData(true, data.getValidUntil(), ownerId), "v2"))); assertTrue(lockProvider.tryLock()); when(mockHeartbeatManager.hasActiveHeartbeat()) - .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active, - // return true. - .thenReturn(false); // when try to set lock to expire we will assert no active heartbeat as a - // precondition. + .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active, return true. + .thenReturn(false); // when try to set lock to expire we will assert no active heartbeat as a precondition. lockProvider.unlock(); assertNull(lockProvider.getLock()); lockProvider.unlock(); From 6b2f1f0eb72a9db7f0adcf7bf5293bc24b8646c3 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Mon, 7 Apr 2025 21:42:36 -0700 Subject: [PATCH 04/25] more checkstyle --- .../lock/TestConditionalWriteLockProvider.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java index 83a3a1aaf2233..7e42583b08da7 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java @@ -45,7 +45,13 @@ import java.util.function.Supplier; import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; @@ -389,7 +395,7 @@ void testUnlockSucceedsAndReentrancy() { new StorageLockFile(new StorageLockData(true, data.getValidUntil(), ownerId), "v2"))); assertTrue(lockProvider.tryLock()); when(mockHeartbeatManager.hasActiveHeartbeat()) - .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active, return true. + .thenReturn(true) // when we try to stop the heartbeat we will check if heartbeat is active return true. .thenReturn(false); // when try to set lock to expire we will assert no active heartbeat as a precondition. lockProvider.unlock(); assertNull(lockProvider.getLock()); From 2cda6c8fba4855568d2ebaf89114ffc75f554500 Mon Sep 17 00:00:00 2001 From: Alex R Date: Tue, 8 Apr 2025 16:00:09 -0700 Subject: [PATCH 05/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java index eda78add50cc4..a85a2e8c760e0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -42,7 +42,7 @@ public class ConditionalWriteLockConfig extends HoodieConfig { .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( "For conditional write based lock provider, the optional URI where lock files are written. " - + "Must be the same filesystem as the table path and should conditional writes. " + + "Must be the same filesystem as the table path and should support conditional writes. " + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); From 96a09780dea413fa2d57023ea6abd3d172c2343c Mon Sep 17 00:00:00 2001 From: Alex R Date: Tue, 8 Apr 2025 16:00:34 -0700 Subject: [PATCH 06/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java index a85a2e8c760e0..eb893a5b476da 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -47,7 +47,7 @@ public class ConditionalWriteLockConfig extends HoodieConfig { + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); public static final ConfigProperty LOCK_VALIDITY_TIMEOUT_MS = ConfigProperty - .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "lock_validity_timeout_ms") + .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "validity.timeout.ms") .defaultValue(TimeUnit.MINUTES.toMillis(5)) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) From 697da63f4e67b1340a9bf4ca267e8dad79f9e161 Mon Sep 17 00:00:00 2001 From: Alex R Date: Tue, 8 Apr 2025 16:00:42 -0700 Subject: [PATCH 07/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java index eb893a5b476da..439dfbc9d2f65 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -62,7 +62,7 @@ public class ConditionalWriteLockConfig extends HoodieConfig { .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( - "For storage based conditional write lock provider, the amount of time to wait before renewing the lock. Defaults to 30 seconds."); + "For storage-based conditional write lock provider, the amount of time to wait before renewing the lock. Defaults to 30 seconds."); public long getLockValidityTimeoutMs() { return getLong(LOCK_VALIDITY_TIMEOUT_MS); From 39d9430762558108e40357943c5182e482008dd8 Mon Sep 17 00:00:00 2001 From: Alex R Date: Tue, 8 Apr 2025 16:00:57 -0700 Subject: [PATCH 08/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java index 439dfbc9d2f65..5e73553b50b9b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -41,7 +41,8 @@ public class ConditionalWriteLockConfig extends HoodieConfig { .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( - "For conditional write based lock provider, the optional URI where lock files are written. " + "For storage-based lock provider, the optional URI where lock files are written. " + + "For example, if `/lock/location` is specified, `/lock/location/` is used as the lock file, where the lock file name `lock_file_name` is determined based on the table's base path. " + "Must be the same filesystem as the table path and should support conditional writes. " + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); From a4658928e402985d04a78d7ff1f4de59d91264a3 Mon Sep 17 00:00:00 2001 From: Alex R Date: Tue, 8 Apr 2025 16:01:42 -0700 Subject: [PATCH 09/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java index c08055b94481a..852cb7dd90082 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java @@ -83,7 +83,7 @@ public class ConditionalWriteLockProvider implements LockProvider Date: Tue, 8 Apr 2025 16:01:51 -0700 Subject: [PATCH 10/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java index 852cb7dd90082..8db5c8c2f1607 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java @@ -95,7 +95,7 @@ public class ConditionalWriteLockProvider implements LockProvider Date: Tue, 8 Apr 2025 16:02:00 -0700 Subject: [PATCH 11/25] Update hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java Co-authored-by: Y Ethan Guo --- .../client/transaction/lock/ConditionalWriteLockProvider.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java index 8db5c8c2f1607..0ad25ca77329d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java @@ -106,7 +106,6 @@ public class ConditionalWriteLockProvider implements LockProvider Date: Tue, 8 Apr 2025 16:03:24 -0700 Subject: [PATCH 12/25] Apply suggestions from code review Co-authored-by: Y Ethan Guo --- .../lock/ConditionalWriteLockConfig.java | 8 ++++---- .../lock/ConditionalWriteLockProvider.java | 13 ++++++------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java index 5e73553b50b9b..10551397af2c1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java @@ -53,12 +53,12 @@ public class ConditionalWriteLockConfig extends HoodieConfig { .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( - "For storage based conditional write lock provider, the amount of time each new lock is valid for." - + "The lock provider will attempt to renew its lock until it successful extends the lock lease period" + "For storage-based lock provider, the amount of time each new lock is valid for. " + + "The lock provider will attempt to renew its lock until it successfully extends the lock lease period " + "or the validity timeout is reached."); public static final ConfigProperty HEARTBEAT_POLL_MS = ConfigProperty - .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "heartbeat_poll_ms") + .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "heartbeat.poll.ms") .defaultValue(TimeUnit.SECONDS.toMillis(30)) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) @@ -75,7 +75,7 @@ public long getHeartbeatPollMs() { public String getHudiTableBasePath() { // Required!! - return getString(BASE_PATH.key()); + return getString(BASE_PATH); } public String getLocksLocation() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java index 0ad25ca77329d..62b04fb460c72 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java @@ -82,7 +82,6 @@ public class ConditionalWriteLockProvider implements LockProvider Date: Tue, 8 Apr 2025 18:57:28 -0700 Subject: [PATCH 13/25] respond to feedback and refactor models --- ...onfig.java => StorageBasedLockConfig.java} | 60 +++---- ...der.java => StorageBasedLockProvider.java} | 165 ++++++++---------- .../lock/ConditionalWriteLockConfigTest.java | 112 ------------ .../lock/StorageBasedLockConfigTest.java | 114 ++++++++++++ ...java => TestStorageBasedLockProvider.java} | 32 ++-- .../hudi/common/fs/TestStorageSchemes.java | 6 +- .../apache/hudi/storage/StorageSchemes.java | 87 ++++----- 7 files changed, 285 insertions(+), 291 deletions(-) rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/{ConditionalWriteLockConfig.java => StorageBasedLockConfig.java} (64%) rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/{ConditionalWriteLockProvider.java => StorageBasedLockProvider.java} (82%) delete mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java rename hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/{TestConditionalWriteLockProvider.java => TestStorageBasedLockProvider.java} (96%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java similarity index 64% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java index 10551397af2c1..b149ab156fd7f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java @@ -28,15 +28,15 @@ import java.util.concurrent.TimeUnit; import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; -import static org.apache.hudi.client.transaction.lock.ConditionalWriteLockProvider.DEFAULT_TABLE_LOCK_FILE_NAME; +import static org.apache.hudi.client.transaction.lock.StorageBasedLockProvider.DEFAULT_TABLE_LOCK_FILE_NAME; import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; -public class ConditionalWriteLockConfig extends HoodieConfig { +public class StorageBasedLockConfig extends HoodieConfig { private static final String SINCE_VERSION_1_0_2 = "1.0.2"; - private static final String CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX - + "conditional_write."; + private static final String STORAGE_BASED_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX + + "storage"; public static final ConfigProperty LOCK_INTERNAL_STORAGE_LOCATION = ConfigProperty - .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "locks_location") + .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "locks_location") .defaultValue("") .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) @@ -47,34 +47,34 @@ public class ConditionalWriteLockConfig extends HoodieConfig { + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); - public static final ConfigProperty LOCK_VALIDITY_TIMEOUT_MS = ConfigProperty - .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "validity.timeout.ms") - .defaultValue(TimeUnit.MINUTES.toMillis(5)) + public static final ConfigProperty LOCK_VALIDITY_TIMEOUT = ConfigProperty + .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "validity.timeout") + .defaultValue(TimeUnit.MINUTES.toSeconds(5)) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( - "For storage-based lock provider, the amount of time each new lock is valid for. " + "For storage-based lock provider, the amount of time in seconds each new lock is valid for. " + "The lock provider will attempt to renew its lock until it successfully extends the lock lease period " + "or the validity timeout is reached."); - public static final ConfigProperty HEARTBEAT_POLL_MS = ConfigProperty - .key(CONDITIONAL_WRITE_LOCK_PROPERTY_PREFIX + "heartbeat.poll.ms") - .defaultValue(TimeUnit.SECONDS.toMillis(30)) + public static final ConfigProperty HEARTBEAT_POLL = ConfigProperty + .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "heartbeat.poll") + .defaultValue(TimeUnit.SECONDS.toSeconds(30)) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( - "For storage-based conditional write lock provider, the amount of time to wait before renewing the lock. Defaults to 30 seconds."); + "For storage-based conditional write lock provider, the amount of time in seconds to wait before renewing the lock." + + "Defaults to 30 seconds."); - public long getLockValidityTimeoutMs() { - return getLong(LOCK_VALIDITY_TIMEOUT_MS); + public long getLockValidityTimeout() { + return getLong(LOCK_VALIDITY_TIMEOUT); } - public long getHeartbeatPollMs() { - return getLong(HEARTBEAT_POLL_MS); + public long getHeartbeatPoll() { + return getLong(HEARTBEAT_POLL); } public String getHudiTableBasePath() { - // Required!! return getString(BASE_PATH); } @@ -83,14 +83,14 @@ public String getLocksLocation() { } public static class Builder { - private final ConditionalWriteLockConfig lockConfig = new ConditionalWriteLockConfig(); + private final StorageBasedLockConfig lockConfig = new StorageBasedLockConfig(); - public ConditionalWriteLockConfig build() { - lockConfig.setDefaults(ConditionalWriteLockConfig.class.getName()); + public StorageBasedLockConfig build() { + lockConfig.setDefaults(StorageBasedLockConfig.class.getName()); return lockConfig; } - public ConditionalWriteLockConfig.Builder fromProperties(TypedProperties props) { + public StorageBasedLockConfig.Builder fromProperties(TypedProperties props) { lockConfig.getProps().putAll(props); checkRequiredProps(); return this; @@ -98,7 +98,7 @@ public ConditionalWriteLockConfig.Builder fromProperties(TypedProperties props) private void checkRequiredProps() { String notExistsMsg = " does not exist!"; - if (Boolean.FALSE.equals(lockConfig.contains(BASE_PATH.key()))) { + if (!lockConfig.contains(BASE_PATH)) { throw new IllegalArgumentException(BASE_PATH.key() + notExistsMsg); } if (lockConfig.getStringOrDefault(LOCK_INTERNAL_STORAGE_LOCATION) @@ -106,19 +106,19 @@ private void checkRequiredProps() { throw new IllegalArgumentException( LOCK_INTERNAL_STORAGE_LOCATION.key() + " cannot start with the hudi table base path."); } - if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) + if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT) < lockConfig.getLongOrDefault(HEARTBEAT_POLL) * 3) { throw new IllegalArgumentException( - LOCK_VALIDITY_TIMEOUT_MS.key() + " should be more than triple " + HEARTBEAT_POLL_MS.key()); + LOCK_VALIDITY_TIMEOUT.key() + " should be more than triple " + HEARTBEAT_POLL.key()); } - if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT_MS) < 5000) { + if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT) < 5) { throw new IllegalArgumentException( - LOCK_VALIDITY_TIMEOUT_MS.key() + " should be greater than or equal to 5 seconds."); + LOCK_VALIDITY_TIMEOUT.key() + " should be greater than or equal to 5 seconds."); } - if (lockConfig.getLongOrDefault(HEARTBEAT_POLL_MS) < 1000) { + if (lockConfig.getLongOrDefault(HEARTBEAT_POLL) < 1) { throw new IllegalArgumentException( - HEARTBEAT_POLL_MS.key() + " should be greater than or equal to 1 second."); + HEARTBEAT_POLL.key() + " should be greater than or equal to 1 second."); } } } -} \ No newline at end of file +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java similarity index 82% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 62b04fb460c72..2f9277692adba 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -27,7 +27,7 @@ import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.lock.LockState; -import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.VisibleForTesting; @@ -48,7 +48,6 @@ import java.net.URI; import java.net.URISyntaxException; -import java.util.Map; import java.util.Objects; import java.util.Properties; import java.util.UUID; @@ -62,12 +61,18 @@ import static org.apache.hudi.common.lock.LockState.RELEASING; import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; +/** + * A distributed filesystem storage based lock provider. This {@link LockProvider} implementation + * leverages conditional writes to ensure transactional consistency for multi-writer scenarios. + * The underlying storage client interface {@link StorageLock} is pluggable so it can be implemented for any + * filesystem which supports conditional writes. + */ @ThreadSafe -public class ConditionalWriteLockProvider implements LockProvider { +public class StorageBasedLockProvider implements LockProvider { public static final String DEFAULT_TABLE_LOCK_FILE_NAME = "table_lock"; // How long to wait before retrying lock acquisition in blocking calls. - private static final long DEFAULT_LOCK_ACQUISITION_BUFFER = 1000; + private static final long DEFAULT_LOCK_ACQUISITION_BUFFER_MS = 1000; // Maximum expected clock drift between two nodes. // This is similar idea as SkewAdjustingTimeGenerator. // In reality, within a single cloud provider all nodes share the same ntp @@ -79,26 +84,15 @@ public class ConditionalWriteLockProvider implements LockProvider SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS = CollectionUtils.createImmutableMap( - Pair.of(StorageSchemes.GCS.getScheme(), GCS_LOCK_SERVICE_CLASS_NAME), - Pair.of(StorageSchemes.S3A.getScheme(), S3_LOCK_SERVICE_CLASS_NAME), - Pair.of(StorageSchemes.S3.getScheme(), S3_LOCK_SERVICE_CLASS_NAME)); + private static final Logger LOGGER = LoggerFactory.getLogger(StorageBasedLockProvider.class); - @VisibleForTesting Logger logger; // The lock service implementation which interacts with storage private final StorageLock lockService; - private final long heartbeatIntervalMs; - private final long lockValidityMs; + private final long heartbeatInterval; + private final long lockValidity; private final String ownerId; private final String lockFilePath; private final String bucketName; @@ -111,7 +105,7 @@ public class ConditionalWriteLockProvider implements LockProvider conf) { - ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() + public StorageBasedLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration conf) { + StorageBasedLockConfig config = new StorageBasedLockConfig.Builder() .fromProperties(lockConfiguration.getConfig()).build(); - heartbeatIntervalMs = config.getHeartbeatPollMs(); - lockValidityMs = config.getLockValidityTimeoutMs(); + heartbeatInterval = config.getHeartbeatPoll(); + lockValidity = config.getLockValidityTimeout(); - // Determine if the provided locks location is relative. String configuredLocksLocation = config.getLocksLocation(); // If not configured, recalculate the locks location as .hoodie/.locks; @@ -151,10 +144,10 @@ public ConditionalWriteLockProvider(final LockConfiguration lockConfiguration, f lockFilePath = buildLockObjectPath(folderName, fileName); ownerId = UUID.randomUUID().toString(); - this.logger = DEFAULT_LOGGER; + this.logger = LOGGER; this.heartbeatManager = new LockProviderHeartbeatManager( ownerId, - heartbeatIntervalMs, + heartbeatInterval, this::renewLock); try { @@ -178,26 +171,26 @@ private URI parseURI(String location) { } private static @NotNull String getLockServiceClassName(String scheme) { - if (SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.containsKey(scheme) - && StorageSchemes.isConditionalWriteSupported(scheme)) { - return SUPPORTED_LOCK_SERVICE_IMPLEMENTATIONS.get(scheme); + Option schemeOptional = StorageSchemes.getStorageLockImplementationIfExists(scheme); + if (schemeOptional.isPresent()) { + return schemeOptional.get().getStorageLockClass(); } else { throw new HoodieNotSupportedException("No implementation of StorageLock supports this scheme: " + scheme); } } @VisibleForTesting - ConditionalWriteLockProvider( - int heartbeatIntervalMs, - int lockValidityMs, + StorageBasedLockProvider( + int heartbeatInterval, + int lockValidity, String bucketName, String lockFilePath, String ownerId, HeartbeatManager heartbeatManager, StorageLock lockService, Logger logger) { - this.heartbeatIntervalMs = heartbeatIntervalMs; - this.lockValidityMs = lockValidityMs; + this.heartbeatInterval = heartbeatInterval; + this.lockValidity = lockValidity; this.bucketName = bucketName; this.lockFilePath = lockFilePath; this.heartbeatManager = heartbeatManager; @@ -228,13 +221,8 @@ public boolean tryLock(long time, TimeUnit unit) { if (tryLock()) { return true; } - logger.debug( - LOCK_STATE_LOGGER_MSG, - ownerId, - lockFilePath, - Thread.currentThread(), - ACQUIRING); - Thread.sleep(DEFAULT_LOCK_ACQUISITION_BUFFER); + logDebugLockState(ACQUIRING); + Thread.sleep(DEFAULT_LOCK_ACQUISITION_BUFFER_MS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new HoodieLockException(generateLockStateMessage(LockState.FAILED_TO_ACQUIRE), e); @@ -276,9 +264,9 @@ private synchronized boolean isLockStillValid(StorageLockFile lock) { */ @Override public synchronized boolean tryLock() { - assertHeartBeatManagerExists(); + assertHeartbeatManagerExists(); assertUnclosed(); - logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), ACQUIRING); + logDebugLockState(ACQUIRING); if (actuallyHoldsLock()) { // Supports reentrant locks return true; @@ -293,13 +281,7 @@ public synchronized boolean tryLock() { Pair latestLock = this.lockService.readCurrentLockFile(); if (latestLock.getLeft() == LockGetResult.UNKNOWN_ERROR) { - logger.info( - LOCK_STATE_LOGGER_MSG_WITH_INFO, - ownerId, - lockFilePath, - Thread.currentThread(), - FAILED_TO_ACQUIRE, - "Failed to get the latest lock status"); + logInfoLockState(FAILED_TO_ACQUIRE, "Failed to get the latest lock status"); // We were not able to determine whether a lock was present. return false; } @@ -307,28 +289,17 @@ public synchronized boolean tryLock() { if (latestLock.getLeft() == LockGetResult.SUCCESS && isLockStillValid(latestLock.getRight())) { String msg = String.format("Lock already held by %s", latestLock.getRight().getOwner()); // Lock held by others. - logger.info( - LOCK_STATE_LOGGER_MSG_WITH_INFO, - ownerId, - lockFilePath, - Thread.currentThread(), - FAILED_TO_ACQUIRE, - msg); + logInfoLockState(FAILED_TO_ACQUIRE, msg); return false; } // Try to acquire the lock - StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId); + StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + lockValidity, ownerId); Pair lockUpdateStatus = this.lockService.tryCreateOrUpdateLockFile( newLockData, latestLock.getLeft() == LockGetResult.NOT_EXISTS ? null : latestLock.getRight()); if (lockUpdateStatus.getLeft() != LockUpdateResult.SUCCESS) { // failed to acquire the lock, indicates concurrent contention - logger.info( - LOCK_STATE_LOGGER_MSG, - ownerId, - lockFilePath, - Thread.currentThread(), - FAILED_TO_ACQUIRE); + logInfoLockState(FAILED_TO_ACQUIRE); return false; } this.setLock(lockUpdateStatus.getRight()); @@ -344,23 +315,12 @@ public synchronized boolean tryLock() { // Precondition "no active heartbeat" is checked previously, so when // startHeartbeatForThread returns false, // we are confident no heartbeat thread is running. - logger.error( - LOCK_STATE_LOGGER_MSG_WITH_INFO, - ownerId, - lockFilePath, - Thread.currentThread(), - RELEASING, - "We were unable to start the heartbeat!"); + logErrorLockState(RELEASING, "We were unable to start the heartbeat!"); tryExpireCurrentLock(); return false; } - logger.info( - LOCK_STATE_LOGGER_MSG, - ownerId, - lockFilePath, - Thread.currentThread(), - ACQUIRED); + logInfoLockState(ACQUIRED); return true; } @@ -393,7 +353,7 @@ private boolean actuallyHoldsLock() { * * @return {@code true} if this provider has a non-null lock object, * {@code false} otherwise - * @see ConditionalWriteLockProvider#actuallyHoldsLock() + * @see StorageBasedLockProvider#actuallyHoldsLock() */ private boolean believesLockMightBeHeld() { return this.getLock() != null; @@ -404,7 +364,7 @@ private boolean believesLockMightBeHeld() { */ @Override public synchronized void unlock() { - assertHeartBeatManagerExists(); + assertHeartbeatManagerExists(); if (!believesLockMightBeHeld()) { return; } @@ -449,7 +409,7 @@ private synchronized boolean tryExpireCurrentLock() { // broken function precondition. throw new HoodieLockException("Must stop heartbeat before expire lock file"); } - logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASING); + logDebugLockState(RELEASING); // Upload metadata that will unlock this lock. Pair result = this.lockService.tryCreateOrUpdateLockFileWithRetry( () -> new StorageLockData(true, this.getLock().getValidUntil(), ownerId), @@ -459,16 +419,16 @@ private synchronized boolean tryExpireCurrentLock() { switch (result.getLeft()) { case UNKNOWN_ERROR: // Here we do not know the state of the lock. - logger.error(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), FAILED_TO_RELEASE); + logErrorLockState(FAILED_TO_RELEASE, "Lock state is unknown."); return false; case SUCCESS: - logger.info(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); + logInfoLockState(RELEASED); setLock(null); return true; case ACQUIRED_BY_OTHERS: - // As we are confident no lock is held by itself, clean up the cached lock - // object. - logger.warn(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), RELEASED); + // As we are confident no lock is held by itself, clean up the cached lock object. + // However this is an edge case, so warn. + logWarnLockState(RELEASED, "lock should not have been acquired by others."); setLock(null); return true; default: @@ -480,6 +440,7 @@ private synchronized boolean tryExpireCurrentLock() { * Renews (heartbeats) the current lock if we are the holder, it forcefully set * the expiration flag * to false and the lock expiration time to a later time in the future. + * @return True if we successfully renewed the lock, false if not. */ @VisibleForTesting protected synchronized boolean renewLock() { @@ -491,7 +452,7 @@ protected synchronized boolean renewLock() { return false; } - long oldExpirationMs = getLock().getValidUntil(); + long oldExpiration = getLock().getValidUntil(); // Attempt conditional update, extend lock. There are 3 cases: // 1. Happy case: lock has not expired yet, we extend the lease to a longer // period. @@ -504,7 +465,7 @@ protected synchronized boolean renewLock() { // prevents further data corruption by // letting someone else acquire the lock. Pair currentLock = this.lockService.tryCreateOrUpdateLockFileWithRetry( - () -> new StorageLockData(false, System.currentTimeMillis() + lockValidityMs, ownerId), + () -> new StorageLockData(false, System.currentTimeMillis() + lockValidity, ownerId), getLock(), LOCK_UPSERT_RETRY_COUNT); switch (currentLock.getLeft()) { @@ -523,7 +484,7 @@ protected synchronized boolean renewLock() { // Only positive outcome this.setLock(currentLock.getRight()); logger.info("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}.", - ownerId, oldExpirationMs - System.currentTimeMillis(), lockFilePath); + ownerId, oldExpiration - System.currentTimeMillis(), lockFilePath); // Let heartbeat continue to renew lock lease again later. return true; default: @@ -543,8 +504,8 @@ protected synchronized boolean renewLock() { * Method to calculate whether a timestamp from a distributed source has * definitively occurred yet. */ - protected boolean isCurrentTimeCertainlyOlderThanDistributedTime(long epochMs) { - return System.currentTimeMillis() > epochMs + CLOCK_DRIFT_BUFFER_MS; + protected boolean isCurrentTimeCertainlyOlderThanDistributedTime(long epoch) { + return System.currentTimeMillis() > epoch + CLOCK_DRIFT_BUFFER_MS; } private String buildLockObjectPath(String lockFolderName, String lockTableFileName) { @@ -576,4 +537,28 @@ private String generateLockStateMessage(LockState state) { return String.format("Owner %s: Lock file path %s, Thread %s, Conditional Write lock state %s", ownerId, lockFilePath, threadName, state.toString()); } + + private static final String LOCK_STATE_LOGGER_MSG = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}"; + private static final String LOCK_STATE_LOGGER_MSG_WITH_INFO = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}, {}"; + + private void logDebugLockState(LockState state) { + logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), state); + } + + private void logInfoLockState(LockState state) { + logger.info(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), state); + } + + private void logInfoLockState(LockState state, String msg) { + logger.info(LOCK_STATE_LOGGER_MSG_WITH_INFO, ownerId, lockFilePath, Thread.currentThread(), state, msg); + } + + private void logWarnLockState(LockState state, String msg) { + logger.warn(LOCK_STATE_LOGGER_MSG_WITH_INFO, ownerId, lockFilePath, Thread.currentThread(), state, msg); + } + + private void logErrorLockState(LockState state, String msg) { + logger.warn(LOCK_STATE_LOGGER_MSG_WITH_INFO, ownerId, lockFilePath, Thread.currentThread(), state, msg); + + } } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java deleted file mode 100644 index d4cc9b3311071..0000000000000 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/ConditionalWriteLockConfigTest.java +++ /dev/null @@ -1,112 +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.client.transaction.lock; - -import org.apache.hudi.common.config.TypedProperties; - -import org.junit.jupiter.api.Test; - -import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; - -class ConditionalWriteLockConfigTest { - - @Test - void testDefaultValues() { - TypedProperties props = new TypedProperties(); - props.setProperty(BASE_PATH.key(), "s3://hudi-bucket/table/basepath"); - - ConditionalWriteLockConfig.Builder builder = new ConditionalWriteLockConfig.Builder(); - ConditionalWriteLockConfig config = builder - .fromProperties(props) - .build(); - - assertEquals("", config.getLocksLocation()); - assertEquals(5 * 60 * 1000, config.getLockValidityTimeoutMs(), "Default lock validity should be 5 minutes"); - assertEquals(30 * 1000, config.getHeartbeatPollMs(), "Default heartbeat poll time should be 30 seconds"); - } - - @Test - void testCustomValues() { - TypedProperties props = new TypedProperties(); - props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "10000"); - props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - - ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() - .fromProperties(props) - .build(); - - assertEquals("s3://bucket/path/locks", config.getLocksLocation()); - assertEquals(120000, config.getLockValidityTimeoutMs()); - assertEquals(10000, config.getHeartbeatPollMs()); - assertEquals("/hudi/table/basepath", config.getHudiTableBasePath()); - } - - @Test - void testMissingRequiredProperties() { - TypedProperties props = new TypedProperties(); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "120000"); - ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); - - IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, - () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(BASE_PATH.key())); - props.setProperty(BASE_PATH.key(), "s3://bucket/path/locks"); - props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); - assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - } - - @Test - void testTimeThresholds() { - TypedProperties props = new TypedProperties(); - props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "3000"); - ConditionalWriteLockConfig.Builder propsBuilder = new ConditionalWriteLockConfig.Builder(); - - IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, - () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "4999"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); - exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key())); - props.setProperty(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); - props.setProperty(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "999"); - exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key())); - } - - @Test - void testBucketPathValidation() { - TypedProperties props = new TypedProperties(); - props.setProperty(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "invalid/path"); - props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - - ConditionalWriteLockConfig config = new ConditionalWriteLockConfig.Builder() - .fromProperties(props) - .build(); - - assertEquals("invalid/path", config.getLocksLocation(), "Locks location should not modify invalid inputs"); - } -} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java new file mode 100644 index 0000000000000..de67720ad1cff --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction.lock; + +import org.apache.hudi.common.config.TypedProperties; + +import org.junit.jupiter.api.Test; + +import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class StorageBasedLockConfigTest { + + @Test + void testDefaultValues() { + // Asserts that the defaults are correct + TypedProperties props = new TypedProperties(); + props.setProperty(BASE_PATH.key(), "s3://hudi-bucket/table/basepath"); + + StorageBasedLockConfig.Builder builder = new StorageBasedLockConfig.Builder(); + StorageBasedLockConfig config = builder + .fromProperties(props) + .build(); + + assertEquals("", config.getLocksLocation()); + assertEquals(5 * 60, config.getLockValidityTimeout(), "Default lock validity should be 5 minutes"); + assertEquals(30, config.getHeartbeatPoll(), "Default heartbeat poll time should be 30 seconds"); + } + + @Test + void testCustomValues() { + // Testing that custom values which differ from defaults can be read properly + TypedProperties props = new TypedProperties(); + props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); + props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "120"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "10"); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + + StorageBasedLockConfig config = new StorageBasedLockConfig.Builder() + .fromProperties(props) + .build(); + + assertEquals("s3://bucket/path/locks", config.getLocksLocation()); + assertEquals(120, config.getLockValidityTimeout()); + assertEquals(10, config.getHeartbeatPoll()); + assertEquals("/hudi/table/basepath", config.getHudiTableBasePath()); + } + + @Test + void testBasePathPropertiesValidation() { + // Tests that validations around the base path are present. + TypedProperties props = new TypedProperties(); + props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "120000"); + StorageBasedLockConfig.Builder propsBuilder = new StorageBasedLockConfig.Builder(); + + // Missing base path + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, + () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(BASE_PATH.key())); + props.setProperty(BASE_PATH.key(), "s3://bucket/path/locks"); + // Ensure we cannot write to the same lock location as the base path. + props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); + assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + // Ensure we cannot write to the metadata directory. + props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks/.hoodie/.metadata"); + assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + // Ensure we cannot write to a partition. + props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks/partition"); + assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + // Ensure we do not throw an exception. + props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/other-path"); + propsBuilder.fromProperties(props); + } + + @Test + void testTimeThresholds() { + // Ensure that validations which restrict the time-based inputs are working. + TypedProperties props = new TypedProperties(); + props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "5"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "3"); + StorageBasedLockConfig.Builder propsBuilder = new StorageBasedLockConfig.Builder(); + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, + () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key())); + props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "4"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "1"); + exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key())); + props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "5"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "0"); + exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); + assertTrue(exception.getMessage().contains(StorageBasedLockConfig.HEARTBEAT_POLL.key())); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java similarity index 96% rename from hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java index 7e42583b08da7..18e8ab7f35bae 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestConditionalWriteLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java @@ -68,8 +68,8 @@ /** * Unit test class for ConditionalWriteLockProvider */ -class TestConditionalWriteLockProvider { - private ConditionalWriteLockProvider lockProvider; +class TestStorageBasedLockProvider { + private StorageBasedLockProvider lockProvider; private StorageLock mockLockService; private HeartbeatManager mockHeartbeatManager; private Logger mockLogger; @@ -82,9 +82,9 @@ void setupLockProvider() { mockHeartbeatManager = mock(HeartbeatManager.class); mockLogger = mock(Logger.class); when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); - lockProvider = spy(new ConditionalWriteLockProvider( - 1000, - DEFAULT_LOCK_VALIDITY_MS, + lockProvider = spy(new StorageBasedLockProvider( + 1, + DEFAULT_LOCK_VALIDITY_MS / 1000, "my-bucket", "gs://bucket/lake/db/tbl-default", ownerId, @@ -105,7 +105,7 @@ void testUnsupportedLockStorageLocation() { LockConfiguration lockConf = new LockConfiguration(props); StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + () -> new StorageBasedLockProvider(lockConf, storageConf)); assertTrue(ex.getCause().getMessage().contains("No implementation of StorageLock supports this scheme")); } @@ -113,13 +113,13 @@ void testUnsupportedLockStorageLocation() { void testValidLockStorageLocation() { TypedProperties props = new TypedProperties(); props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); - props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/locks"); + props.put(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/locks"); LockConfiguration lockConf = new LockConfiguration(props); StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + () -> new StorageBasedLockProvider(lockConf, storageConf)); assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); } @@ -127,13 +127,13 @@ void testValidLockStorageLocation() { void testInvalidLockStorageLocation() { TypedProperties props = new TypedProperties(); props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); - props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), + props.put(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/lake/db/tbl-default/.hoodie/.metadata"); LockConfiguration lockConf = new LockConfiguration(props); StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - assertThrows(IllegalArgumentException.class, () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + assertThrows(IllegalArgumentException.class, () -> new StorageBasedLockProvider(lockConf, storageConf)); } @ParameterizedTest @@ -147,23 +147,23 @@ void testNonExistentWriteServiceWithDefaults(String tableBasePathString) { StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + () -> new StorageBasedLockProvider(lockConf, storageConf)); assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); } @Test void testInvalidLocksLocationForWriteService() { TypedProperties props = new TypedProperties(); - props.put(ConditionalWriteLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "not a uri"); + props.put(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "not a uri"); props.put(BASE_PATH.key(), "gs://bucket/lake/db/tbl-default"); - props.put(ConditionalWriteLockConfig.LOCK_VALIDITY_TIMEOUT_MS.key(), "5000"); - props.put(ConditionalWriteLockConfig.HEARTBEAT_POLL_MS.key(), "1000"); + props.put(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "5"); + props.put(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "1"); LockConfiguration lockConf = new LockConfiguration(props); StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); HoodieLockException ex = assertThrows(HoodieLockException.class, - () -> new ConditionalWriteLockProvider(lockConf, storageConf)); + () -> new StorageBasedLockProvider(lockConf, storageConf)); Throwable cause = ex.getCause(); assertNotNull(cause); assertInstanceOf(URISyntaxException.class, cause); @@ -252,7 +252,7 @@ void testTryLockFailsFromOwnerMismatch() { .thenReturn(Pair.of(LockUpdateResult.SUCCESS, returnedLockFile)); HoodieLockException ex = assertThrows(HoodieLockException.class, () -> lockProvider.tryLock()); - assertTrue(ex.getMessage().contains("Owners do not match!")); + assertTrue(ex.getMessage().contains("Owners do not match")); } @Test diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index 0245d612f4b57..3d285dfcac8bb 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -39,9 +39,9 @@ public void testStorageSchemes() { assertTrue(StorageSchemes.isSchemeSupported("afs")); assertFalse(StorageSchemes.isSchemeSupported("s2")); - assertTrue(StorageSchemes.isConditionalWriteSupported("s3")); - assertTrue(StorageSchemes.isConditionalWriteSupported("s3a")); - assertTrue(StorageSchemes.isConditionalWriteSupported("gs")); + assertTrue(StorageSchemes.getStorageLockImplementationIfExists("s3").isPresent()); + assertTrue(StorageSchemes.getStorageLockImplementationIfExists("s3a").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("gs").isPresent()); assertTrue(StorageSchemes.isAtomicCreationSupported("file")); assertTrue(StorageSchemes.isAtomicCreationSupported("hdfs")); diff --git a/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java b/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java index 62f9aa146b321..aabb50104eb45 100644 --- a/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java +++ b/hudi-io/src/main/java/org/apache/hudi/storage/StorageSchemes.java @@ -19,6 +19,9 @@ package org.apache.hudi.storage; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; + import java.util.Arrays; import java.util.Set; import java.util.HashSet; @@ -28,66 +31,66 @@ */ public enum StorageSchemes { // Local filesystem - FILE("file", false, true, false), + FILE("file", false, true, null), // Hadoop File System - HDFS("hdfs", false, true, false), + HDFS("hdfs", false, true, null), // Baidu Advanced File System - AFS("afs", null, null, false), + AFS("afs", null, null, null), // Mapr File System - MAPRFS("maprfs", null, null, false), + MAPRFS("maprfs", null, null, null), // Apache Ignite FS - IGNITE("igfs", null, null, false), + IGNITE("igfs", null, null, null), // AWS S3 - S3A("s3a", true, null, true), - S3("s3", true, null, true), + S3A("s3a", true, null, "org.apache.hudi.aws.transaction.lock.S3StorageLock"), + S3("s3", true, null, "org.apache.hudi.aws.transaction.lock.S3StorageLock"), // Google Cloud Storage - GCS("gs", true, null, true), + GCS("gs", true, null, null), // Azure WASB - WASB("wasb", null, null, false), - WASBS("wasbs", null, null, false), + WASB("wasb", null, null, null), + WASBS("wasbs", null, null, null), // Azure ADLS - ADL("adl", null, null, false), + ADL("adl", null, null, null), // Azure ADLS Gen2 - ABFS("abfs", null, null, false), - ABFSS("abfss", null, null, false), + ABFS("abfs", null, null, null), + ABFSS("abfss", null, null, null), // Aliyun OSS - OSS("oss", null, null, false), + OSS("oss", null, null, null), // View FS for federated setups. If federating across cloud stores, then append // support is false // View FS support atomic creation - VIEWFS("viewfs", null, true, false), + VIEWFS("viewfs", null, true, null), // ALLUXIO - ALLUXIO("alluxio", null, null, false), + ALLUXIO("alluxio", null, null, null), // Tencent Cloud Object Storage - COSN("cosn", null, null, false), + COSN("cosn", null, null, null), // Tencent Cloud HDFS - CHDFS("ofs", null, null, false), + CHDFS("ofs", null, null, null), // Tencent Cloud CacheFileSystem - GOOSEFS("gfs", null, null, false), + GOOSEFS("gfs", null, null, null), // Databricks file system - DBFS("dbfs", null, null, false), + DBFS("dbfs", null, null, null), // IBM Cloud Object Storage - COS("cos", null, null, false), + COS("cos", null, null, null), // Huawei Cloud Object Storage - OBS("obs", null, null, false), + OBS("obs", null, null, null), // Kingsoft Standard Storage ks3 - KS3("ks3", null, null, false), + KS3("ks3", null, null, null), // Netease Object Storage nos - NOS("nos", null, null, false), + NOS("nos", null, null, null), // JuiceFileSystem - JFS("jfs", null, null, false), + JFS("jfs", null, null, null), // Baidu Object Storage - BOS("bos", null, null, false), + BOS("bos", null, null, null), // Oracle Cloud Infrastructure Object Storage - OCI("oci", null, null, false), + OCI("oci", null, null, null), // Volcengine Object Storage - TOS("tos", null, null, false), + TOS("tos", null, null, null), // Volcengine Cloud HDFS - CFS("cfs", null, null, false), + CFS("cfs", null, null, null), // Aliyun Apsara File Storage for HDFS - DFS("dfs", false, true, false), + DFS("dfs", false, true, null), // Hopsworks File System - HOPSFS("hopsfs", false, true, false); + HOPSFS("hopsfs", false, true, null); // list files may bring pressure to storage with centralized meta service like HDFS. // when we want to get only part of files under a directory rather than all files, use getStatus may be more friendly than listStatus. @@ -99,17 +102,17 @@ public enum StorageSchemes { private final Boolean isWriteTransactional; // null for uncertain if dfs support atomic create&delete, please update this for each FS private final Boolean supportAtomicCreation; - private final Boolean supportsConditionalWrite; + private final String storageLockClass; StorageSchemes( String scheme, Boolean isWriteTransactional, Boolean supportAtomicCreation, - Boolean supportsConditionalWrite) { + String storageLockClass) { this.scheme = scheme; this.isWriteTransactional = isWriteTransactional; this.supportAtomicCreation = supportAtomicCreation; - this.supportsConditionalWrite = supportsConditionalWrite; + this.storageLockClass = storageLockClass; } public String getScheme() { @@ -124,8 +127,12 @@ public boolean isAtomicCreationSupported() { return supportAtomicCreation != null && supportAtomicCreation; } - public boolean isConditionalWriteSupported() { - return supportsConditionalWrite != null && supportsConditionalWrite; + public boolean implementsStorageLock() { + return !StringUtils.isNullOrEmpty(storageLockClass); + } + + public String getStorageLockClass() { + return storageLockClass; } public static boolean isSchemeSupported(String scheme) { @@ -155,11 +162,11 @@ public static boolean isListStatusFriendly(String scheme) { return LIST_STATUS_FRIENDLY_SCHEMES.contains(scheme); } - public static boolean isConditionalWriteSupported(String scheme) { + public static Option getStorageLockImplementationIfExists(String scheme) { if (!isSchemeSupported(scheme)) { throw new IllegalArgumentException("Unsupported scheme :" + scheme); } - return Arrays.stream(StorageSchemes.values()) - .anyMatch(s -> s.isConditionalWriteSupported() && s.scheme.equals(scheme)); + return Option.fromJavaOptional(Arrays.stream(StorageSchemes.values()) + .filter(s -> s.implementsStorageLock() && s.scheme.equals(scheme)).findFirst()); } -} \ No newline at end of file +} From 5519af2eaf792d5cd1457b8d4499d875b4a866d2 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Tue, 8 Apr 2025 18:58:43 -0700 Subject: [PATCH 14/25] update config --- .../hudi/client/transaction/lock/StorageBasedLockConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java index b149ab156fd7f..9d976c9a37e98 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java @@ -42,7 +42,8 @@ public class StorageBasedLockConfig extends HoodieConfig { .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( "For storage-based lock provider, the optional URI where lock files are written. " - + "For example, if `/lock/location` is specified, `/lock/location/` is used as the lock file, where the lock file name `lock_file_name` is determined based on the table's base path. " + + "For example, if `/lock/location` is specified, `/lock/location/` is used as the lock file," + + "where the lock file name `lock_file_name` is determined based on the table's base path. " + "Must be the same filesystem as the table path and should support conditional writes. " + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); From 69548013f5cc6e4e495cd6ac3c5c62c180c93c89 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Wed, 9 Apr 2025 21:01:37 -0700 Subject: [PATCH 15/25] rename class --- ...BasedLockConfigTest.java => TestStorageBasedLockConfig.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/{StorageBasedLockConfigTest.java => TestStorageBasedLockConfig.java} (99%) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockConfig.java similarity index 99% rename from hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockConfig.java index de67720ad1cff..1e393086a99a1 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfigTest.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockConfig.java @@ -27,7 +27,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -class StorageBasedLockConfigTest { +class TestStorageBasedLockConfig { @Test void testDefaultValues() { From d5310ed36892caa4f50a320b1024010f33feadd6 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Wed, 9 Apr 2025 22:04:51 -0700 Subject: [PATCH 16/25] conditional writes rename --- .../lock/StorageBasedLockProvider.java | 18 +++++++++--------- .../models/LockProviderHeartbeatManager.java | 2 +- .../lock/models/StorageLockFile.java | 10 +++++----- .../lock/TestStorageBasedLockProvider.java | 2 +- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 2f9277692adba..fb9be4581f5d9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -112,11 +112,11 @@ private synchronized void setLock(StorageLockFile lockObj) { } /** - * Default constructor for ConditionalWriteLockProvider, required by LockManager + * Default constructor for StorageBasedLockProvider, required by LockManager * to instantiate it using reflection. * * @param lockConfiguration The lock configuration, should be transformable into - * ConditionalWriteLockConfig + * StorageBasedLockConfig * @param conf Storage config, ignored. */ public StorageBasedLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration conf) { @@ -197,7 +197,7 @@ private URI parseURI(String location) { this.lockService = lockService; this.ownerId = ownerId; this.logger = logger; - logger.debug("Instantiating new Conditional Write LP, owner: {}", ownerId); + logger.debug("Instantiating new Storage Based LP, owner: {}", ownerId); } // ----------------------------------------- @@ -345,11 +345,11 @@ private boolean actuallyHoldsLock() { *

* A non-null lock object indicates that this provider has previously * **successfully** acquired a lock via - * ConditionalWriteLockProvider##lock and has not yet **successfully** released - * it via ConditionalWriteLockProvider#unlock(). + * StorageBasedLockProvider##lock and has not yet **successfully** released + * it via StorageBasedLockProvider#unlock(). * It is merely an indicator that the lock might be held by this provider. To * truly certify we are the owner of the lock, - * ConditionalWriteLockProvider#actuallyHoldsLock should be used. + * StorageBasedLockProvider#actuallyHoldsLock should be used. * * @return {@code true} if this provider has a non-null lock object, * {@code false} otherwise @@ -534,12 +534,12 @@ private String slugifyLockFolderFromBasePath(String basePathKey) { private String generateLockStateMessage(LockState state) { String threadName = Thread.currentThread().getName(); - return String.format("Owner %s: Lock file path %s, Thread %s, Conditional Write lock state %s", ownerId, + return String.format("Owner %s: Lock file path %s, Thread %s, Storage based lock state %s", ownerId, lockFilePath, threadName, state.toString()); } - private static final String LOCK_STATE_LOGGER_MSG = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}"; - private static final String LOCK_STATE_LOGGER_MSG_WITH_INFO = "Owner {}: Lock file path {}, Thread {}, Conditional Write lock state {}, {}"; + private static final String LOCK_STATE_LOGGER_MSG = "Owner {}: Lock file path {}, Thread {}, Storage based lock state {}"; + private static final String LOCK_STATE_LOGGER_MSG_WITH_INFO = "Owner {}: Lock file path {}, Thread {}, Storage based lock state {}, {}"; private void logDebugLockState(LockState state) { logger.debug(LOCK_STATE_LOGGER_MSG, ownerId, lockFilePath, Thread.currentThread(), state); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/LockProviderHeartbeatManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/LockProviderHeartbeatManager.java index 4beba39dc9579..be73bd7cb2c0e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/LockProviderHeartbeatManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/LockProviderHeartbeatManager.java @@ -35,7 +35,7 @@ /** * LockProviderHeartbeatManager is a helper class which handles the scheduling and stopping of heartbeat - * tasks. This is intended for use with the conditional write lock provider, which requires + * tasks. This is intended for use with the storage based lock provider, which requires * a separate thread to spawn and renew the lock repeatedly. * It should be responsible for the entire lifecycle of the heartbeat task. * Importantly, a new instance should be created for each lock provider. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/StorageLockFile.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/StorageLockFile.java index 2a045f31aefc3..dfccf23d67edd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/StorageLockFile.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/models/StorageLockFile.java @@ -35,7 +35,7 @@ public class StorageLockFile { private final StorageLockData data; private final String versionId; - // Get a custom object mapper. See ConditionalWriteLockData for required properties. + // Get a custom object mapper. See StorageLockData for required properties. private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() // This allows us to add new properties down the line. .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES) @@ -43,7 +43,7 @@ public class StorageLockFile { .enable(DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES); /** - * Initializes a ConditionalWriteLockFile using the data and unique versionId. + * Initializes a StorageLockFile using the data and unique versionId. * * @param data The data in the lock file. * @param versionId The version of this lock file. Used to ensure consistency through conditional writes. @@ -56,11 +56,11 @@ public StorageLockFile(StorageLockData data, String versionId) { } /** - * Factory method to load an input stream into a ConditionalWriteLockFile. + * Factory method to load an input stream into a StorageLockFile. * * @param inputStream The input stream of the JSON content. * @param versionId The unique version identifier for the lock file. - * @return A new instance of ConditionalWriteLockFile. + * @return A new instance of StorageLockFile. * @throws HoodieIOException If deserialization fails. */ public static StorageLockFile createFromStream(InputStream inputStream, String versionId) { @@ -68,7 +68,7 @@ public static StorageLockFile createFromStream(InputStream inputStream, String v StorageLockData data = OBJECT_MAPPER.readValue(inputStream, StorageLockData.class); return new StorageLockFile(data, versionId); } catch (IOException e) { - throw new HoodieIOException("Failed to deserialize JSON content into ConditionalWriteLockData", e); + throw new HoodieIOException("Failed to deserialize JSON content into StorageLockData", e); } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java index 18e8ab7f35bae..9345dfd35ab3f 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java @@ -66,7 +66,7 @@ import static org.mockito.Mockito.when; /** - * Unit test class for ConditionalWriteLockProvider + * Unit test class for StorageBasedLockProvider */ class TestStorageBasedLockProvider { private StorageBasedLockProvider lockProvider; From e081f835713cec7b7f77bc29cda45fbf25e07159 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Thu, 10 Apr 2025 15:35:51 -0700 Subject: [PATCH 17/25] add shutdown hook --- .../lock/StorageBasedLockProvider.java | 18 +++++++- .../lock/TestStorageBasedLockProvider.java | 43 +++++++++++++++++++ 2 files changed, 60 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index fb9be4581f5d9..a5c83f37d78aa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -97,6 +97,7 @@ public class StorageBasedLockProvider implements LockProvider { private final String lockFilePath; private final String bucketName; private final HeartbeatManager heartbeatManager; + private transient Thread shutdownThread = null; @GuardedBy("this") private StorageLockFile currentLockObj = null; @@ -158,7 +159,8 @@ public StorageBasedLockProvider(final LockConfiguration lockConfiguration, final } catch (Throwable e) { throw new HoodieLockException("Failed to load and initialize StorageLock", e); } - + shutdownThread = new Thread(() -> shutdown(true)); + Runtime.getRuntime().addShutdownHook(shutdownThread); logger.info("Instantiated new storage-based lock provider, owner: {}, lockfilePath: {}", ownerId, lockFilePath); } @@ -198,6 +200,8 @@ private URI parseURI(String location) { this.ownerId = ownerId; this.logger = logger; logger.debug("Instantiating new Storage Based LP, owner: {}", ownerId); + shutdownThread = new Thread(() -> shutdown(true)); + Runtime.getRuntime().addShutdownHook(shutdownThread); } // ----------------------------------------- @@ -234,6 +238,18 @@ public boolean tryLock(long time, TimeUnit unit) { @Override public synchronized void close() { + shutdown(false); + } + + private synchronized void shutdown(boolean fromShutdownHook) { + if (fromShutdownHook) { + if (isClosed || !actuallyHoldsLock()) { + return; + } + // Ensure we release the lock before shutdown + } else { + Runtime.getRuntime().removeShutdownHook(shutdownThread); + } try { this.unlock(); } catch (Exception e) { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java index 9345dfd35ab3f..ba8ed0a427754 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java @@ -35,8 +35,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentMatchers; import org.slf4j.Logger; +import java.lang.reflect.Method; import java.net.URISyntaxException; import java.util.UUID; import java.util.concurrent.CountDownLatch; @@ -61,6 +63,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -561,6 +564,46 @@ void testCloseWithErrorForHeartbeatManager() throws Exception { assertNull(lockProvider.getLock()); } + @Test + public void testShutdownHookViaReflection() throws Exception { + when(mockLockService.readCurrentLockFile()).thenReturn(Pair.of(LockGetResult.NOT_EXISTS, null)); + StorageLockData data = new StorageLockData(false, System.currentTimeMillis() + DEFAULT_LOCK_VALIDITY_MS, ownerId); + StorageLockFile realLockFile = new StorageLockFile(data, "v1"); + when(mockLockService.tryCreateOrUpdateLockFile(any(), isNull())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + when(mockHeartbeatManager.startHeartbeatForThread(any())).thenReturn(true); + + boolean acquired = lockProvider.tryLock(); + assertTrue(acquired); + assertEquals(realLockFile, lockProvider.getLock()); + verify(mockLockService, atLeastOnce()).tryCreateOrUpdateLockFile(any(), any()); + + when(mockLockService.tryCreateOrUpdateLockFileWithRetry(ArgumentMatchers.any(), eq(realLockFile), anyLong())) + .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); + + // Mock shutdown + Method shutdownMethod = lockProvider.getClass().getDeclaredMethod("shutdown", boolean.class); + shutdownMethod.setAccessible(true); + shutdownMethod.invoke(lockProvider, true); + + // Verify that the expected shutdown behaviors occurred. + assertNull(lockProvider.getLock()); + verify(mockLockService, atLeastOnce()).close(); + verify(mockHeartbeatManager, atLeastOnce()).close(); + } + + @Test + public void testShutdownHookWhenNoLockPresent() throws Exception { + // Now, when calling shutdown(true), the method should immediately return. + Method shutdownMethod = lockProvider.getClass().getDeclaredMethod("shutdown", boolean.class); + shutdownMethod.setAccessible(true); + shutdownMethod.invoke(lockProvider, true); + + // Verify that unlock or close methods are NOT invoked, or adjust expectations accordingly. + verify(mockLockService, never()).close(); + verify(mockHeartbeatManager, never()).close(); + } + public static class StubStorageLock implements StorageLock { public StubStorageLock(String arg1, String arg2, String arg3) { // No-op constructor for reflection From 2d3784b7aed77a40a0ccccd7638874f5af802e60 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Thu, 10 Apr 2025 15:42:28 -0700 Subject: [PATCH 18/25] refactor configs based on feedback --- .../client/transaction/lock/StorageBasedLockProvider.java | 1 + .../transaction/lock => config}/StorageBasedLockConfig.java | 4 ++-- .../client/transaction/lock/TestStorageBasedLockProvider.java | 1 + .../lock => config}/TestStorageBasedLockConfig.java | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/{client/transaction/lock => config}/StorageBasedLockConfig.java (98%) rename hudi-client/hudi-client-common/src/test/java/org/apache/hudi/{client/transaction/lock => config}/TestStorageBasedLockConfig.java (99%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index a5c83f37d78aa..9bad6013d0d0e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.hash.HashID; +import org.apache.hudi.config.StorageBasedLockConfig; import org.apache.hudi.exception.HoodieLockException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.storage.StorageConfiguration; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java similarity index 98% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java index 9d976c9a37e98..dad2e379aedfb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.client.transaction.lock; +package org.apache.hudi.config; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; @@ -34,7 +34,7 @@ public class StorageBasedLockConfig extends HoodieConfig { private static final String SINCE_VERSION_1_0_2 = "1.0.2"; private static final String STORAGE_BASED_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX - + "storage"; + + "storage."; public static final ConfigProperty LOCK_INTERNAL_STORAGE_LOCATION = ConfigProperty .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "locks_location") .defaultValue("") diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java index ba8ed0a427754..13399c156c156 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.StorageBasedLockConfig; import org.apache.hudi.exception.HoodieLockException; import org.apache.hudi.storage.StorageConfiguration; diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java similarity index 99% rename from hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockConfig.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java index 1e393086a99a1..ddb31587ae5ae 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.client.transaction.lock; +package org.apache.hudi.config; import org.apache.hudi.common.config.TypedProperties; From 45d1b482b85ab3d3ffef6e132241507a48e349aa Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Thu, 10 Apr 2025 17:55:34 -0700 Subject: [PATCH 19/25] cleanup lp initialization based on feedback --- .../lock/StorageBasedLockProvider.java | 168 +++++++----------- .../hudi/config/StorageBasedLockConfig.java | 49 ++--- .../lock/TestStorageBasedLockProvider.java | 48 ++--- .../config/TestStorageBasedLockConfig.java | 48 ++--- .../hudi/common/fs/TestStorageSchemes.java | 21 +++ 5 files changed, 128 insertions(+), 206 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 9bad6013d0d0e..467aefb6282b0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -25,14 +25,14 @@ import org.apache.hudi.client.transaction.lock.models.LockProviderHeartbeatManager; import org.apache.hudi.client.transaction.lock.models.LockUpdateResult; import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.lock.LockState; +import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.common.util.hash.HashID; import org.apache.hudi.config.StorageBasedLockConfig; import org.apache.hudi.exception.HoodieLockException; import org.apache.hudi.exception.HoodieNotSupportedException; @@ -48,11 +48,11 @@ import javax.annotation.concurrent.ThreadSafe; import java.net.URI; -import java.net.URISyntaxException; import java.util.Objects; import java.util.Properties; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import static org.apache.hudi.common.lock.LockState.ACQUIRED; import static org.apache.hudi.common.lock.LockState.ACQUIRING; @@ -87,18 +87,17 @@ public class StorageBasedLockProvider implements LockProvider { private static final Logger LOGGER = LoggerFactory.getLogger(StorageBasedLockProvider.class); - Logger logger; + // Use for testing + private final Logger logger; // The lock service implementation which interacts with storage private final StorageLock lockService; - private final long heartbeatInterval; - private final long lockValidity; + private final long validitySeconds; private final String ownerId; private final String lockFilePath; - private final String bucketName; private final HeartbeatManager heartbeatManager; - private transient Thread shutdownThread = null; + private final transient Thread shutdownThread; @GuardedBy("this") private StorageLockFile currentLockObj = null; @@ -122,55 +121,26 @@ private synchronized void setLock(StorageLockFile lockObj) { * @param conf Storage config, ignored. */ public StorageBasedLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration conf) { - StorageBasedLockConfig config = new StorageBasedLockConfig.Builder() - .fromProperties(lockConfiguration.getConfig()).build(); - heartbeatInterval = config.getHeartbeatPoll(); - lockValidity = config.getLockValidityTimeout(); - - String configuredLocksLocation = config.getLocksLocation(); - - // If not configured, recalculate the locks location as .hoodie/.locks; - // otherwise (the lock location is configured), the configuration location is used as the folder - // to which the lock file is written to, and the lock file name is determined by the table's base path - String locksLocation = StringUtils.isNullOrEmpty(configuredLocksLocation) - ? String.format("%s%s%s", config.getHudiTableBasePath(), StoragePath.SEPARATOR, LOCKS_FOLDER_NAME) - : configuredLocksLocation; - - URI uri = parseURI(locksLocation); - bucketName = uri.getHost(); // For most schemes, the bucket/container is the host. - String folderName = uri.getPath(); // Path after the bucket/container. - - String fileName = StringUtils.isNullOrEmpty(configuredLocksLocation) - ? DEFAULT_TABLE_LOCK_FILE_NAME - : slugifyLockFolderFromBasePath(config.getHudiTableBasePath()); - - lockFilePath = buildLockObjectPath(folderName, fileName); - ownerId = UUID.randomUUID().toString(); - this.logger = LOGGER; - this.heartbeatManager = new LockProviderHeartbeatManager( - ownerId, - heartbeatInterval, - this::renewLock); - - try { - this.lockService = (StorageLock) ReflectionUtils.loadClass( - getLockServiceClassName(uri.getScheme()), - new Class[] { String.class, String.class, String.class, Properties.class }, - new Object[] { ownerId, bucketName, lockFilePath, lockConfiguration.getConfig() }); - } catch (Throwable e) { - throw new HoodieLockException("Failed to load and initialize StorageLock", e); - } - shutdownThread = new Thread(() -> shutdown(true)); - Runtime.getRuntime().addShutdownHook(shutdownThread); - logger.info("Instantiated new storage-based lock provider, owner: {}, lockfilePath: {}", ownerId, lockFilePath); + this( + UUID.randomUUID().toString(), + lockConfiguration.getConfig(), + LockProviderHeartbeatManager::new, + tryLoadLockService(), + LOGGER); } - private URI parseURI(String location) { - try { - return new URI(location); - } catch (URISyntaxException e) { - throw new HoodieLockException("Unable to parse locks location as a URI: " + location, e); - } + private static Functions.Function3 tryLoadLockService() + { + return (ownerId, lockFilePath, lockConfig) -> { + try { + return (StorageLock) ReflectionUtils.loadClass( + getLockServiceClassName(new URI(lockFilePath).getScheme()), + new Class[]{String.class, String.class, String.class, Properties.class}, + new Object[]{ownerId, lockFilePath, lockConfig}); + } catch (Throwable e) { + throw new HoodieLockException("Failed to load and initialize StorageLock", e); + } + }; } private static @NotNull String getLockServiceClassName(String scheme) { @@ -184,25 +154,22 @@ private URI parseURI(String location) { @VisibleForTesting StorageBasedLockProvider( - int heartbeatInterval, - int lockValidity, - String bucketName, - String lockFilePath, String ownerId, - HeartbeatManager heartbeatManager, - StorageLock lockService, + TypedProperties properties, + Functions.Function3, HeartbeatManager> heartbeatManagerLoader, + Functions.Function3 lockServiceLoader, Logger logger) { - this.heartbeatInterval = heartbeatInterval; - this.lockValidity = lockValidity; - this.bucketName = bucketName; - this.lockFilePath = lockFilePath; - this.heartbeatManager = heartbeatManager; - this.lockService = lockService; + StorageBasedLockConfig config = new StorageBasedLockConfig.Builder().fromProperties(properties).build(); + long heartbeatPollSeconds = config.getHeartbeatPollSeconds(); + this.validitySeconds = config.getValiditySeconds(); + this.lockFilePath = String.format("%s%s%s", config.getHudiTableBasePath(), StoragePath.SEPARATOR, LOCKS_FOLDER_NAME); + this.heartbeatManager = heartbeatManagerLoader.apply(ownerId, heartbeatPollSeconds, this::renewLock); + this.lockService = lockServiceLoader.apply(ownerId, lockFilePath, properties); this.ownerId = ownerId; this.logger = logger; - logger.debug("Instantiating new Storage Based LP, owner: {}", ownerId); shutdownThread = new Thread(() -> shutdown(true)); Runtime.getRuntime().addShutdownHook(shutdownThread); + logger.info("Instantiated new storage-based lock provider, owner: {}, lockfilePath: {}", ownerId, lockFilePath); } // ----------------------------------------- @@ -223,10 +190,10 @@ public boolean tryLock(long time, TimeUnit unit) { while (System.nanoTime() < deadlineNanos) { try { + logDebugLockState(ACQUIRING); if (tryLock()) { return true; } - logDebugLockState(ACQUIRING); Thread.sleep(DEFAULT_LOCK_ACQUISITION_BUFFER_MS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -244,10 +211,12 @@ public synchronized void close() { private synchronized void shutdown(boolean fromShutdownHook) { if (fromShutdownHook) { - if (isClosed || !actuallyHoldsLock()) { - return; + // Try to expire the lock from the shutdown hook. + if (!isClosed && actuallyHoldsLock()) { + tryExpireCurrentLock(true); } - // Ensure we release the lock before shutdown + // Do not execute any further actions + return; } else { Runtime.getRuntime().removeShutdownHook(shutdownThread); } @@ -311,7 +280,7 @@ public synchronized boolean tryLock() { } // Try to acquire the lock - StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + lockValidity, ownerId); + StorageLockData newLockData = new StorageLockData(false, System.currentTimeMillis() + validitySeconds, ownerId); Pair lockUpdateStatus = this.lockService.tryCreateOrUpdateLockFile( newLockData, latestLock.getLeft() == LockGetResult.NOT_EXISTS ? null : latestLock.getRight()); if (lockUpdateStatus.getLeft() != LockUpdateResult.SUCCESS) { @@ -333,7 +302,7 @@ public synchronized boolean tryLock() { // startHeartbeatForThread returns false, // we are confident no heartbeat thread is running. logErrorLockState(RELEASING, "We were unable to start the heartbeat!"); - tryExpireCurrentLock(); + tryExpireCurrentLock(false); return false; } @@ -394,7 +363,7 @@ public synchronized void unlock() { } // Then expire the current lock. - believesNoLongerHoldsLock &= tryExpireCurrentLock(); + believesNoLongerHoldsLock &= tryExpireCurrentLock(false); if (!believesNoLongerHoldsLock) { throw new HoodieLockException(generateLockStateMessage(FAILED_TO_RELEASE)); } @@ -415,24 +384,31 @@ private void assertUnclosed() { /** * Tries to expire the currently held lock. - * + * @param fromShutdownHook Whether we are attempting best effort quick unlock from shutdown hook. * @return True if we were successfully able to upload an expired lock. */ - private synchronized boolean tryExpireCurrentLock() { + private synchronized boolean tryExpireCurrentLock(boolean fromShutdownHook) { // It does not make sense to have heartbeat alive extending the lock lease while // here we are trying // to expire the lock. - if (heartbeatManager.hasActiveHeartbeat()) { + if (!fromShutdownHook && heartbeatManager.hasActiveHeartbeat()) { // broken function precondition. throw new HoodieLockException("Must stop heartbeat before expire lock file"); } logDebugLockState(RELEASING); // Upload metadata that will unlock this lock. - Pair result = this.lockService.tryCreateOrUpdateLockFileWithRetry( - () -> new StorageLockData(true, this.getLock().getValidUntil(), ownerId), - this.getLock(), - // Keep retrying for the normal validity time. - LOCK_UPSERT_RETRY_COUNT); + StorageLockData expiredLockData = new StorageLockData(true, this.getLock().getValidUntil(), ownerId); + Pair result; + if (fromShutdownHook) { + // Only try once for shutdown hook, then return immediately + result = this.lockService.tryCreateOrUpdateLockFile(expiredLockData, this.getLock()); + } else { + result = this.lockService.tryCreateOrUpdateLockFileWithRetry( + () -> expiredLockData, + this.getLock(), + // Keep retrying for the normal validity time. + LOCK_UPSERT_RETRY_COUNT); + } switch (result.getLeft()) { case UNKNOWN_ERROR: // Here we do not know the state of the lock. @@ -482,7 +458,7 @@ protected synchronized boolean renewLock() { // prevents further data corruption by // letting someone else acquire the lock. Pair currentLock = this.lockService.tryCreateOrUpdateLockFileWithRetry( - () -> new StorageLockData(false, System.currentTimeMillis() + lockValidity, ownerId), + () -> new StorageLockData(false, System.currentTimeMillis() + validitySeconds, ownerId), getLock(), LOCK_UPSERT_RETRY_COUNT); switch (currentLock.getLeft()) { @@ -525,30 +501,6 @@ protected boolean isCurrentTimeCertainlyOlderThanDistributedTime(long epoch) { return System.currentTimeMillis() > epoch + CLOCK_DRIFT_BUFFER_MS; } - private String buildLockObjectPath(String lockFolderName, String lockTableFileName) { - // Normalize inputs by removing trailing slashes - // We know lockTableFileName has already been parsed. - if (lockFolderName.startsWith("/")) { - lockFolderName = lockFolderName.substring(1); - } - - // Append a slash only if one isn't already present. - return lockFolderName + (lockFolderName.endsWith("/") ? "" : "/") + lockTableFileName + ".json"; - } - - private String slugifyLockFolderFromBasePath(String basePathKey) { - // Remove the prefix once - String cleanedPath = basePathKey.replaceFirst("^(gs://|s3://|s3a://)", ""); - - // Generate the lock name - return cleanedPath - .replaceAll("[/\\\\]+", "-") // Replace slashes with dashes - .replaceAll("[^0-9a-zA-Z_-]", "-") // Replace invalid characters - .toLowerCase() - .substring(Math.max(0, cleanedPath.length() - 40)) // Get last 40 characters - + "_" + HashID.generateXXHashAsString(basePathKey, HashID.Size.BITS_64); - } - private String generateLockStateMessage(LockState state) { String threadName = Thread.currentThread().getName(); return String.format("Owner %s: Lock file path %s, Thread %s, Storage based lock state %s", ownerId, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java index dad2e379aedfb..13096a9a01d30 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java @@ -35,21 +35,9 @@ public class StorageBasedLockConfig extends HoodieConfig { private static final String SINCE_VERSION_1_0_2 = "1.0.2"; private static final String STORAGE_BASED_LOCK_PROPERTY_PREFIX = LockConfiguration.LOCK_PREFIX + "storage."; - public static final ConfigProperty LOCK_INTERNAL_STORAGE_LOCATION = ConfigProperty - .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "locks_location") - .defaultValue("") - .markAdvanced() - .sinceVersion(SINCE_VERSION_1_0_2) - .withDocumentation( - "For storage-based lock provider, the optional URI where lock files are written. " - + "For example, if `/lock/location` is specified, `/lock/location/` is used as the lock file," - + "where the lock file name `lock_file_name` is determined based on the table's base path. " - + "Must be the same filesystem as the table path and should support conditional writes. " - + "By default, writes to " + LOCKS_FOLDER_NAME + Path.SEPARATOR - + DEFAULT_TABLE_LOCK_FILE_NAME + ".json under the table base path."); - public static final ConfigProperty LOCK_VALIDITY_TIMEOUT = ConfigProperty - .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "validity.timeout") + public static final ConfigProperty VALIDITY_TIMEOUT_SECONDS = ConfigProperty + .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "validity.timeout.secs") .defaultValue(TimeUnit.MINUTES.toSeconds(5)) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) @@ -58,8 +46,8 @@ public class StorageBasedLockConfig extends HoodieConfig { + "The lock provider will attempt to renew its lock until it successfully extends the lock lease period " + "or the validity timeout is reached."); - public static final ConfigProperty HEARTBEAT_POLL = ConfigProperty - .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "heartbeat.poll") + public static final ConfigProperty HEARTBEAT_POLL_SECONDS = ConfigProperty + .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "heartbeat.poll.secs") .defaultValue(TimeUnit.SECONDS.toSeconds(30)) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) @@ -67,22 +55,18 @@ public class StorageBasedLockConfig extends HoodieConfig { "For storage-based conditional write lock provider, the amount of time in seconds to wait before renewing the lock." + "Defaults to 30 seconds."); - public long getLockValidityTimeout() { - return getLong(LOCK_VALIDITY_TIMEOUT); + public long getValiditySeconds() { + return getLong(VALIDITY_TIMEOUT_SECONDS); } - public long getHeartbeatPoll() { - return getLong(HEARTBEAT_POLL); + public long getHeartbeatPollSeconds() { + return getLong(HEARTBEAT_POLL_SECONDS); } public String getHudiTableBasePath() { return getString(BASE_PATH); } - public String getLocksLocation() { - return getString(LOCK_INTERNAL_STORAGE_LOCATION); - } - public static class Builder { private final StorageBasedLockConfig lockConfig = new StorageBasedLockConfig(); @@ -102,23 +86,18 @@ private void checkRequiredProps() { if (!lockConfig.contains(BASE_PATH)) { throw new IllegalArgumentException(BASE_PATH.key() + notExistsMsg); } - if (lockConfig.getStringOrDefault(LOCK_INTERNAL_STORAGE_LOCATION) - .startsWith(lockConfig.getHudiTableBasePath())) { - throw new IllegalArgumentException( - LOCK_INTERNAL_STORAGE_LOCATION.key() + " cannot start with the hudi table base path."); - } - if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT) < lockConfig.getLongOrDefault(HEARTBEAT_POLL) + if (lockConfig.getLongOrDefault(VALIDITY_TIMEOUT_SECONDS) < lockConfig.getLongOrDefault(HEARTBEAT_POLL_SECONDS) * 3) { throw new IllegalArgumentException( - LOCK_VALIDITY_TIMEOUT.key() + " should be more than triple " + HEARTBEAT_POLL.key()); + VALIDITY_TIMEOUT_SECONDS.key() + " should be more than triple " + HEARTBEAT_POLL_SECONDS.key()); } - if (lockConfig.getLongOrDefault(LOCK_VALIDITY_TIMEOUT) < 5) { + if (lockConfig.getLongOrDefault(VALIDITY_TIMEOUT_SECONDS) < 5) { throw new IllegalArgumentException( - LOCK_VALIDITY_TIMEOUT.key() + " should be greater than or equal to 5 seconds."); + VALIDITY_TIMEOUT_SECONDS.key() + " should be greater than or equal to 5 seconds."); } - if (lockConfig.getLongOrDefault(HEARTBEAT_POLL) < 1) { + if (lockConfig.getLongOrDefault(HEARTBEAT_POLL_SECONDS) < 1) { throw new IllegalArgumentException( - HEARTBEAT_POLL.key() + " should be greater than or equal to 1 second."); + HEARTBEAT_POLL_SECONDS.key() + " should be greater than or equal to 1 second."); } } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java index 13399c156c156..9e5f4a6d689f8 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/lock/TestStorageBasedLockProvider.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.StorageBasedLockConfig; import org.apache.hudi.exception.HoodieLockException; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.storage.StorageConfiguration; import org.junit.jupiter.api.AfterEach; @@ -36,11 +37,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import org.mockito.ArgumentMatchers; import org.slf4j.Logger; import java.lang.reflect.Method; -import java.net.URISyntaxException; import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -86,14 +85,16 @@ void setupLockProvider() { mockHeartbeatManager = mock(HeartbeatManager.class); mockLogger = mock(Logger.class); when(mockHeartbeatManager.stopHeartbeat(true)).thenReturn(true); + TypedProperties props = new TypedProperties(); + props.put(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "5"); + props.put(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "1"); + props.put(BASE_PATH.key(), "gs://bucket/lake/db/tbl-default"); + lockProvider = spy(new StorageBasedLockProvider( - 1, - DEFAULT_LOCK_VALIDITY_MS / 1000, - "my-bucket", - "gs://bucket/lake/db/tbl-default", ownerId, - mockHeartbeatManager, - mockLockService, + props, + (a,b,c) -> mockHeartbeatManager, + (a,b,c) -> mockLockService, mockLogger)); } @@ -117,7 +118,6 @@ void testUnsupportedLockStorageLocation() { void testValidLockStorageLocation() { TypedProperties props = new TypedProperties(); props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); - props.put(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/locks"); LockConfiguration lockConf = new LockConfiguration(props); StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); @@ -127,19 +127,6 @@ void testValidLockStorageLocation() { assertTrue(ex.getMessage().contains("Failed to load and initialize StorageLock")); } - @Test - void testInvalidLockStorageLocation() { - TypedProperties props = new TypedProperties(); - props.put(BASE_PATH.key(), "s3://bucket/lake/db/tbl-default"); - props.put(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), - "s3://bucket/lake/db/tbl-default/.hoodie/.metadata"); - - LockConfiguration lockConf = new LockConfiguration(props); - StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); - - assertThrows(IllegalArgumentException.class, () -> new StorageBasedLockProvider(lockConf, storageConf)); - } - @ParameterizedTest @ValueSource(strings = { "gs://bucket/lake/db/tbl-default", "s3://bucket/lake/db/tbl-default", "s3a://bucket/lake/db/tbl-default" }) @@ -158,10 +145,9 @@ void testNonExistentWriteServiceWithDefaults(String tableBasePathString) { @Test void testInvalidLocksLocationForWriteService() { TypedProperties props = new TypedProperties(); - props.put(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "not a uri"); props.put(BASE_PATH.key(), "gs://bucket/lake/db/tbl-default"); - props.put(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "5"); - props.put(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "1"); + props.put(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "5"); + props.put(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "1"); LockConfiguration lockConf = new LockConfiguration(props); StorageConfiguration storageConf = HoodieTestUtils.getDefaultStorageConf(); @@ -170,8 +156,7 @@ void testInvalidLocksLocationForWriteService() { () -> new StorageBasedLockProvider(lockConf, storageConf)); Throwable cause = ex.getCause(); assertNotNull(cause); - assertInstanceOf(URISyntaxException.class, cause); - assertTrue(ex.getMessage().contains("Unable to parse locks location as a URI")); + assertInstanceOf(HoodieNotSupportedException.class, cause); } @Test @@ -523,7 +508,7 @@ void testRenewLockSucceeds() { verify(mockLogger).info( eq("Owner {}: Lock renewal successful. The renewal completes {} ms before expiration for lock {}."), - eq(this.ownerId), anyLong(), eq("gs://bucket/lake/db/tbl-default")); + eq(this.ownerId), anyLong(), eq("gs://bucket/lake/db/tbl-default/.hoodie/.locks")); } @Test @@ -579,7 +564,7 @@ public void testShutdownHookViaReflection() throws Exception { assertEquals(realLockFile, lockProvider.getLock()); verify(mockLockService, atLeastOnce()).tryCreateOrUpdateLockFile(any(), any()); - when(mockLockService.tryCreateOrUpdateLockFileWithRetry(ArgumentMatchers.any(), eq(realLockFile), anyLong())) + when(mockLockService.tryCreateOrUpdateLockFile(any(StorageLockData.class), eq(realLockFile))) .thenReturn(Pair.of(LockUpdateResult.SUCCESS, realLockFile)); // Mock shutdown @@ -589,8 +574,9 @@ public void testShutdownHookViaReflection() throws Exception { // Verify that the expected shutdown behaviors occurred. assertNull(lockProvider.getLock()); - verify(mockLockService, atLeastOnce()).close(); - verify(mockHeartbeatManager, atLeastOnce()).close(); + // We do not execute additional actions + verify(mockLockService, never()).close(); + verify(mockHeartbeatManager, never()).close(); } @Test diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java index ddb31587ae5ae..618394ce6877a 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java @@ -40,27 +40,24 @@ void testDefaultValues() { .fromProperties(props) .build(); - assertEquals("", config.getLocksLocation()); - assertEquals(5 * 60, config.getLockValidityTimeout(), "Default lock validity should be 5 minutes"); - assertEquals(30, config.getHeartbeatPoll(), "Default heartbeat poll time should be 30 seconds"); + assertEquals(5 * 60, config.getValiditySeconds(), "Default lock validity should be 5 minutes"); + assertEquals(30, config.getHeartbeatPollSeconds(), "Default heartbeat poll time should be 30 seconds"); } @Test void testCustomValues() { // Testing that custom values which differ from defaults can be read properly TypedProperties props = new TypedProperties(); - props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); - props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "120"); - props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "10"); + props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "120"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "10"); props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); StorageBasedLockConfig config = new StorageBasedLockConfig.Builder() .fromProperties(props) .build(); - assertEquals("s3://bucket/path/locks", config.getLocksLocation()); - assertEquals(120, config.getLockValidityTimeout()); - assertEquals(10, config.getHeartbeatPoll()); + assertEquals(120, config.getValiditySeconds()); + assertEquals(10, config.getHeartbeatPollSeconds()); assertEquals("/hudi/table/basepath", config.getHudiTableBasePath()); } @@ -68,26 +65,13 @@ void testCustomValues() { void testBasePathPropertiesValidation() { // Tests that validations around the base path are present. TypedProperties props = new TypedProperties(); - props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "120000"); + props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "120"); StorageBasedLockConfig.Builder propsBuilder = new StorageBasedLockConfig.Builder(); // Missing base path IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); assertTrue(exception.getMessage().contains(BASE_PATH.key())); - props.setProperty(BASE_PATH.key(), "s3://bucket/path/locks"); - // Ensure we cannot write to the same lock location as the base path. - props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks"); - assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - // Ensure we cannot write to the metadata directory. - props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks/.hoodie/.metadata"); - assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - // Ensure we cannot write to a partition. - props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/path/locks/partition"); - assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - // Ensure we do not throw an exception. - props.setProperty(StorageBasedLockConfig.LOCK_INTERNAL_STORAGE_LOCATION.key(), "s3://bucket/other-path"); - propsBuilder.fromProperties(props); } @Test @@ -95,20 +79,20 @@ void testTimeThresholds() { // Ensure that validations which restrict the time-based inputs are working. TypedProperties props = new TypedProperties(); props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); - props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "5"); - props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "3"); + props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "5"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "3"); StorageBasedLockConfig.Builder propsBuilder = new StorageBasedLockConfig.Builder(); IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key())); - props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "4"); - props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "1"); + assertTrue(exception.getMessage().contains(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key())); + props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "4"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "1"); exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key())); - props.setProperty(StorageBasedLockConfig.LOCK_VALIDITY_TIMEOUT.key(), "5"); - props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL.key(), "0"); + assertTrue(exception.getMessage().contains(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key())); + props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "5"); + props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "0"); exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); - assertTrue(exception.getMessage().contains(StorageBasedLockConfig.HEARTBEAT_POLL.key())); + assertTrue(exception.getMessage().contains(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key())); } } \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index 3d285dfcac8bb..c35dcd238415c 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -42,6 +42,27 @@ public void testStorageSchemes() { assertTrue(StorageSchemes.getStorageLockImplementationIfExists("s3").isPresent()); assertTrue(StorageSchemes.getStorageLockImplementationIfExists("s3a").isPresent()); assertFalse(StorageSchemes.getStorageLockImplementationIfExists("gs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("file").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("hdfs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("afs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("wasb").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("adl").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("abfs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("oss").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("viewfs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("alluxio").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("cosn").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("dbfs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("cos").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("jfs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("bos").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("ks3").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("nos").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("ofs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("oci").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("tos").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("cfs").isPresent()); + assertFalse(StorageSchemes.getStorageLockImplementationIfExists("hopsfs").isPresent()); assertTrue(StorageSchemes.isAtomicCreationSupported("file")); assertTrue(StorageSchemes.isAtomicCreationSupported("hdfs")); From 188735aa084c79db779062568d7c4d2e045ef899 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Thu, 10 Apr 2025 17:58:06 -0700 Subject: [PATCH 20/25] checkstyle --- .../client/transaction/lock/StorageBasedLockProvider.java | 3 +-- .../java/org/apache/hudi/config/StorageBasedLockConfig.java | 4 ---- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 467aefb6282b0..004fcdb504c03 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -129,8 +129,7 @@ public StorageBasedLockProvider(final LockConfiguration lockConfiguration, final LOGGER); } - private static Functions.Function3 tryLoadLockService() - { + private static Functions.Function3 tryLoadLockService() { return (ownerId, lockFilePath, lockConfig) -> { try { return (StorageLock) ReflectionUtils.loadClass( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java index 13096a9a01d30..de3030da791cc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java @@ -23,13 +23,9 @@ import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hadoop.fs.Path; - import java.util.concurrent.TimeUnit; import static org.apache.hudi.common.config.HoodieCommonConfig.BASE_PATH; -import static org.apache.hudi.client.transaction.lock.StorageBasedLockProvider.DEFAULT_TABLE_LOCK_FILE_NAME; -import static org.apache.hudi.common.table.HoodieTableMetaClient.LOCKS_FOLDER_NAME; public class StorageBasedLockConfig extends HoodieConfig { private static final String SINCE_VERSION_1_0_2 = "1.0.2"; From 06d4900a66ddc4d5c1ee08fa5d34a1788955cf29 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Thu, 10 Apr 2025 20:33:23 -0700 Subject: [PATCH 21/25] respond to minor feedback --- .../lock/StorageBasedLockProvider.java | 4 +-- .../hudi/common/fs/TestStorageSchemes.java | 32 +++++-------------- 2 files changed, 10 insertions(+), 26 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 004fcdb504c03..737b05816a64b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -125,11 +125,11 @@ public StorageBasedLockProvider(final LockConfiguration lockConfiguration, final UUID.randomUUID().toString(), lockConfiguration.getConfig(), LockProviderHeartbeatManager::new, - tryLoadLockService(), + getLockService(), LOGGER); } - private static Functions.Function3 tryLoadLockService() { + private static Functions.Function3 getLockService() { return (ownerId, lockFilePath, lockConfig) -> { try { return (StorageLock) ReflectionUtils.loadClass( diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index c35dcd238415c..25fe3e34c6fdc 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -39,30 +39,14 @@ public void testStorageSchemes() { assertTrue(StorageSchemes.isSchemeSupported("afs")); assertFalse(StorageSchemes.isSchemeSupported("s2")); - assertTrue(StorageSchemes.getStorageLockImplementationIfExists("s3").isPresent()); - assertTrue(StorageSchemes.getStorageLockImplementationIfExists("s3a").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("gs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("file").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("hdfs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("afs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("wasb").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("adl").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("abfs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("oss").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("viewfs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("alluxio").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("cosn").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("dbfs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("cos").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("jfs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("bos").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("ks3").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("nos").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("ofs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("oci").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("tos").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("cfs").isPresent()); - assertFalse(StorageSchemes.getStorageLockImplementationIfExists("hopsfs").isPresent()); + for (StorageSchemes scheme : StorageSchemes.values()) { + String schemeName = scheme.getScheme(); + if (scheme.getScheme().startsWith("s3")) { + assertTrue(StorageSchemes.getStorageLockImplementationIfExists(schemeName).isPresent()); + } else { + assertFalse(StorageSchemes.getStorageLockImplementationIfExists(schemeName).isPresent()); + } + } assertTrue(StorageSchemes.isAtomicCreationSupported("file")); assertTrue(StorageSchemes.isAtomicCreationSupported("hdfs")); From 269c57d3bc1e9498d95185a2871b9e5404433dd2 Mon Sep 17 00:00:00 2001 From: Alex Rhee Date: Thu, 10 Apr 2025 22:24:34 -0700 Subject: [PATCH 22/25] convert to milliseconds --- .../hudi/client/transaction/lock/StorageBasedLockProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 737b05816a64b..b8bcfaecef3db 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -162,7 +162,7 @@ private static Functions.Function3 long heartbeatPollSeconds = config.getHeartbeatPollSeconds(); this.validitySeconds = config.getValiditySeconds(); this.lockFilePath = String.format("%s%s%s", config.getHudiTableBasePath(), StoragePath.SEPARATOR, LOCKS_FOLDER_NAME); - this.heartbeatManager = heartbeatManagerLoader.apply(ownerId, heartbeatPollSeconds, this::renewLock); + this.heartbeatManager = heartbeatManagerLoader.apply(ownerId, heartbeatPollSeconds * 1000, this::renewLock); this.lockService = lockServiceLoader.apply(ownerId, lockFilePath, properties); this.ownerId = ownerId; this.logger = logger; From 2d596dea259829f006a6af8c949b7d7205c1b300 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 10 Apr 2025 23:24:25 -0700 Subject: [PATCH 23/25] Address comments --- .../client/transaction/lock/StorageBasedLockProvider.java | 6 +++--- .../java/org/apache/hudi/config/StorageBasedLockConfig.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index b8bcfaecef3db..5f357a6e8052c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -18,12 +18,12 @@ package org.apache.hudi.client.transaction.lock; -import org.apache.hudi.client.transaction.lock.models.StorageLockData; -import org.apache.hudi.client.transaction.lock.models.StorageLockFile; import org.apache.hudi.client.transaction.lock.models.HeartbeatManager; import org.apache.hudi.client.transaction.lock.models.LockGetResult; import org.apache.hudi.client.transaction.lock.models.LockProviderHeartbeatManager; import org.apache.hudi.client.transaction.lock.models.LockUpdateResult; +import org.apache.hudi.client.transaction.lock.models.StorageLockData; +import org.apache.hudi.client.transaction.lock.models.StorageLockFile; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.lock.LockProvider; @@ -529,4 +529,4 @@ private void logErrorLockState(LockState state, String msg) { logger.warn(LOCK_STATE_LOGGER_MSG_WITH_INFO, ownerId, lockFilePath, Thread.currentThread(), state, msg); } -} \ No newline at end of file +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java index de3030da791cc..bc5a52db71bf6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/StorageBasedLockConfig.java @@ -44,12 +44,12 @@ public class StorageBasedLockConfig extends HoodieConfig { public static final ConfigProperty HEARTBEAT_POLL_SECONDS = ConfigProperty .key(STORAGE_BASED_LOCK_PROPERTY_PREFIX + "heartbeat.poll.secs") - .defaultValue(TimeUnit.SECONDS.toSeconds(30)) + .defaultValue(30L) .markAdvanced() .sinceVersion(SINCE_VERSION_1_0_2) .withDocumentation( - "For storage-based conditional write lock provider, the amount of time in seconds to wait before renewing the lock." - + "Defaults to 30 seconds."); + "For storage-based lock provider, the amount of time in seconds to wait before renewing the lock. " + + "Defaults to 30 seconds."); public long getValiditySeconds() { return getLong(VALIDITY_TIMEOUT_SECONDS); From 03f4a6c9adbd96611ecfee0f8b02f0f9e8d7ad4c Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 10 Apr 2025 23:33:25 -0700 Subject: [PATCH 24/25] Address more comment --- .../hudi/client/transaction/lock/StorageBasedLockProvider.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java index 5f357a6e8052c..ba9dfb9a3fad7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/StorageBasedLockProvider.java @@ -526,7 +526,6 @@ private void logWarnLockState(LockState state, String msg) { } private void logErrorLockState(LockState state, String msg) { - logger.warn(LOCK_STATE_LOGGER_MSG_WITH_INFO, ownerId, lockFilePath, Thread.currentThread(), state, msg); - + logger.error(LOCK_STATE_LOGGER_MSG_WITH_INFO, ownerId, lockFilePath, Thread.currentThread(), state, msg); } } From 8f21de848122ea24d128cd742f24bc681ed45c5d Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 10 Apr 2025 23:41:52 -0700 Subject: [PATCH 25/25] Address nits --- .../org/apache/hudi/config/TestStorageBasedLockConfig.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java index 618394ce6877a..3be7e5550b253 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestStorageBasedLockConfig.java @@ -79,6 +79,7 @@ void testTimeThresholds() { // Ensure that validations which restrict the time-based inputs are working. TypedProperties props = new TypedProperties(); props.setProperty(BASE_PATH.key(), "/hudi/table/basepath"); + // Invalid config case: validity timeout is less than triple of heartbeat poll period props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "5"); props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "3"); StorageBasedLockConfig.Builder propsBuilder = new StorageBasedLockConfig.Builder(); @@ -86,13 +87,15 @@ void testTimeThresholds() { IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); assertTrue(exception.getMessage().contains(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key())); + // Invalid config case: validity timeout is less than 5 seconds props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "4"); props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "1"); exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); assertTrue(exception.getMessage().contains(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key())); + // Invalid config case: heartbeat poll period is less than 1 second props.setProperty(StorageBasedLockConfig.VALIDITY_TIMEOUT_SECONDS.key(), "5"); props.setProperty(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key(), "0"); exception = assertThrows(IllegalArgumentException.class, () -> propsBuilder.fromProperties(props)); assertTrue(exception.getMessage().contains(StorageBasedLockConfig.HEARTBEAT_POLL_SECONDS.key())); } -} \ No newline at end of file +}