From bf7f722ab859e30ac65085d4243cffcdcc6df43d Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Wed, 23 Apr 2025 23:08:42 +0300 Subject: [PATCH 01/10] init --- .../election/LeaderElectionListener.java | 5 + .../recipes/election/LeaderElector.java | 199 ++++++++++++ .../recipes/locks/InterProcessLock.java | 25 ++ .../recipes/locks/InterProcessMutex.java | 64 ++++ .../locks/LockAcquireFailedException.java | 28 ++ .../locks/LockAlreadyAcquiredException.java | 11 + .../recipes/locks/LockInternals.java | 305 ++++++++++++++++++ .../locks/LockUpgradeFailedException.java | 12 + .../locks/ReadWriteInterProcessLock.java | 118 +++++++ .../recipes/util/ByteSerializable.java | 7 + .../coordination/recipes/util/Listenable.java | 45 +++ .../recipes/util/ListenableAdder.java | 8 + .../recipes/util/ListenableProvider.java | 29 ++ .../recipes/util/ListenerWrapper.java | 51 +++ .../recipes/util/SerializationException.java | 10 + .../coordination/recipes/util/Serializer.java | 18 ++ .../recipes/util/SessionListenerWrapper.java | 54 ++++ .../recipes/watch/Participant.java | 61 ++++ .../recipes/watch/SemaphoreWatchListener.java | 195 +++++++++++ .../coordination/recipes/watch/WatchData.java | 41 +++ .../CoordinationSessionBaseMockedTest.java | 171 ++++++++++ .../recipes/ReadWriteLockTest.java | 25 ++ 22 files changed, 1482 insertions(+) create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java new file mode 100644 index 000000000..30a102477 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java @@ -0,0 +1,5 @@ +package tech.ydb.coordination.recipes.election; + +public interface LeaderElectionListener { + void takeLeadership() throws Exception; +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java new file mode 100644 index 000000000..c4e65d82f --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java @@ -0,0 +1,199 @@ +package tech.ydb.coordination.recipes.election; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.locks.LockInternals; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableProvider; +import tech.ydb.coordination.recipes.watch.Participant; +import tech.ydb.coordination.recipes.watch.SemaphoreWatchListener; + +import java.io.Closeable; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; + +public class LeaderElector implements Closeable, ListenableProvider { + private static final Logger logger = LoggerFactory.getLogger(LeaderElector.class); + + private final CoordinationClient client; + private final LeaderElectionListener leaderElectionListener; + private final String coordinationNodePath; + private final String semaphoreName; + private final ExecutorService electionExecutor; + private final LockInternals lock; + private final SemaphoreWatchListener semaphoreWatchAdapter; + + private AtomicReference state = new AtomicReference<>(State.STARTED); + private volatile boolean autoRequeue = false; + private volatile boolean isLeader = false; + private Future electionTask = null; + + + private enum State { // TODO: needs third state (CREATED)? + STARTED, + CLOSED + } + + public LeaderElector( + CoordinationClient client, + LeaderElectionListener leaderElectionListener, + String coordinationNodePath, + String semaphoreName + ) { + this(client, leaderElectionListener, coordinationNodePath, semaphoreName, Executors.newSingleThreadExecutor()); + } + + public LeaderElector( + CoordinationClient client, + LeaderElectionListener leaderElectionListener, + String coordinationNodePath, + String semaphoreName, + ExecutorService executorService + ) { + this.client = client; + this.leaderElectionListener = leaderElectionListener; + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = semaphoreName; + this.electionExecutor = executorService; + this.lock = new LockInternals( + client, + coordinationNodePath, + semaphoreName + ); + this.lock.start(); + this.semaphoreWatchAdapter = new SemaphoreWatchListener(lock.getCoordinationSession(), semaphoreName); + semaphoreWatchAdapter.start(); + } + + public boolean isLeader() { + return isLeader; + } + + public synchronized void interruptLeadership() { + Future task = electionTask; + if (task != null) { + task.cancel(true); + } + } + + /** + * Re-queue an attempt for leadership. If this instance is already queued, nothing + * happens and false is returned. If the instance was not queued, it is re-queued and true + * is returned + * + * @return true if re-enqueue was successful + */ + public boolean requeue() { + Preconditions.checkState(state.get() == State.STARTED, "Already closed or not yet started"); + + return enqueueElection(); + } + + public void autoRequeue() { + autoRequeue = true; + } + + private synchronized boolean enqueueElection() { + if (!isQueued() && state.get() == State.STARTED) { + electionTask = electionExecutor.submit(new Callable() { + @Override + public Void call() throws Exception { + try { + doWork(); + } finally { + finishTask(); + } + return null; + } + }); + return true; + } + + return false; + } + + private void doWork() throws Exception { + isLeader = false; + + try { + lock.tryAcquire( + null, + true, + null + ); + isLeader = true; + try { + leaderElectionListener.takeLeadership(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw e; + } catch (Throwable e) { + logger.debug("takeLeadership exception", e); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw e; + } finally { + if (isLeader) { + isLeader = false; + boolean wasInterrupted = Thread.interrupted(); + try { + lock.release(); + } catch (Exception e) { + logger.error("Lock release exception for: " + coordinationNodePath); + } finally { + if (wasInterrupted) { + Thread.currentThread().interrupt(); + } + } + } + } + } + + private synchronized void finishTask() { + electionTask = null; + if (autoRequeue) { // TODO: requeue if critical exception? + enqueueElection(); + } + } + + private boolean isQueued() { + return electionTask != null; + } + + public List getParticipants() { + return semaphoreWatchAdapter.getParticipants(); + } + + public Optional getLeader() { + return semaphoreWatchAdapter.getOwners().stream().findFirst(); + } + + @Override + public synchronized void close() { + Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Already closed"); + + Future task = electionTask; + if (task != null) { + task.cancel(true); + } + + electionTask = null; + electionExecutor.shutdown(); + semaphoreWatchAdapter.close(); + getListenable().clearListeners(); + } + + @Override + public Listenable getListenable() { + return lock.getListenable(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java new file mode 100644 index 000000000..4f66b5a35 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java @@ -0,0 +1,25 @@ +package tech.ydb.coordination.recipes.locks; + +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.util.Listenable; + +import java.time.Duration; + +public interface InterProcessLock extends Listenable { + void acquire() throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException; + + /** + * @return true - if successfully acquired lock, false - if lock waiting time expired + */ + boolean acquire(Duration waitDuration) throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException; + + /** + * @return false if nothing to release + */ + boolean release() throws Exception; + + /** + * @return true if the lock is acquired by a thread in this JVM + */ + boolean isAcquiredInThisProcess(); +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java new file mode 100644 index 000000000..6a3fc6883 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java @@ -0,0 +1,64 @@ +package tech.ydb.coordination.recipes.locks; + +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableProvider; + +import javax.annotation.concurrent.ThreadSafe; +import java.io.Closeable; +import java.time.Duration; +import java.time.Instant; + +@ThreadSafe +public class InterProcessMutex implements InterProcessLock, ListenableProvider, Closeable { + private final LockInternals lockInternals; + + public InterProcessMutex( + CoordinationClient client, + String coordinationNodePath, + String lockName + ) { + lockInternals = new LockInternals(client, coordinationNodePath, lockName); + lockInternals.start(); + } + + @Override + public void acquire() throws Exception { + lockInternals.tryAcquire( + null, + true, + null + ); + } + + @Override + public boolean acquire(Duration waitDuration) throws Exception { + Instant deadline = Instant.now().plus(waitDuration); + return lockInternals.tryAcquire( + deadline, + true, + null + ) != null; + } + + @Override + public boolean release() throws Exception { + return lockInternals.release(); + } + + @Override + public boolean isAcquiredInThisProcess() { + return lockInternals.isAcquired(); + } + + @Override + public Listenable getListenable() { + return null; + } + + @Override + public void close() { + lockInternals.close(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java new file mode 100644 index 000000000..a0c3c7951 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java @@ -0,0 +1,28 @@ +package tech.ydb.coordination.recipes.locks; + +public class LockAcquireFailedException extends RuntimeException { + private final String coordinationNodePath; + private final String semaphoreName; + + public LockAcquireFailedException(String message, String coordinationNodePath, String semaphoreName) { + super("Failed to acquire semaphore=" + semaphoreName + + ", on coordination node=" + coordinationNodePath + + ": '" + message + "'"); + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = semaphoreName; + } + + public LockAcquireFailedException(String coordinationNodePath, String semaphoreName) { + super("Failed to acquire semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath); + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = semaphoreName; + } + + public String getCoordinationNodePath() { + return coordinationNodePath; + } + + public String getSemaphoreName() { + return semaphoreName; + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java new file mode 100644 index 000000000..8b127d062 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java @@ -0,0 +1,11 @@ +package tech.ydb.coordination.recipes.locks; + +public class LockAlreadyAcquiredException extends LockAcquireFailedException { + public LockAlreadyAcquiredException(String coordinationNodePath, String semaphoreName) { + super( + "Lock=" + semaphoreName + " on path=" + coordinationNodePath + " is already acquired", + coordinationNodePath, + semaphoreName + ); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java new file mode 100644 index 000000000..10749120a --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java @@ -0,0 +1,305 @@ +package tech.ydb.coordination.recipes.locks; + +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.SemaphoreLease; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableProvider; +import tech.ydb.coordination.recipes.util.SessionListenerWrapper; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.core.Result; +import tech.ydb.core.Status; +import tech.ydb.core.StatusCode; + +import javax.annotation.concurrent.ThreadSafe; +import java.io.Closeable; +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; + +@ThreadSafe +public class LockInternals implements ListenableProvider, Closeable { + private static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(30); + private static final Logger logger = LoggerFactory.getLogger(LockInternals.class); + + private final String coordinationNodePath; + private final String semaphoreName; + private final CoordinationSession session; + private final SessionListenerWrapper sessionListenerWrapper; + + private CompletableFuture sessionConnectionTask = null; + private volatile LeaseData leaseData = null; // TODO: needs to be volatile? + + public static class LeaseData { + private final SemaphoreLease processLease; + private final boolean isExclusive; + + public LeaseData(SemaphoreLease processLease, boolean isExclusive) { + this.processLease = processLease; + this.isExclusive = isExclusive; + } + + public boolean isExclusive() { + return isExclusive; + } + + public SemaphoreLease getProcessLease() { + return processLease; + } + + @Override + public String toString() { + return "LeaseData{" + + "processLease=" + processLease + + ", isExclusive=" + isExclusive + + '}'; + } + } + + public LockInternals( + CoordinationClient client, + String coordinationNodePath, + String lockName + ) { + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = lockName; + this.session = client.createSession(coordinationNodePath); + this.sessionListenerWrapper = new SessionListenerWrapper(session); + } + + public void start() { + this.sessionConnectionTask = session.connect().thenApply(status -> { + logger.debug("Session connection status: {}", status); + return status; + }); + + Consumer listener = state -> { + switch (state) { + case RECONNECTED: { + logger.debug("Session RECONNECTED"); + reconnect(); + break; + } + case CLOSED: { + logger.debug("Session CLOSED, releasing lock"); + release(); + break; + } + case LOST: { + logger.debug("Session LOST, releasing lock"); + release(); + break; + } + } + }; + + session.addStateListener(listener); + } + + private CoordinationSession connectedSession() { + if (sessionConnectionTask == null) { + throw new IllegalStateException("Not started yet"); + } + try { + sessionConnectionTask.get().expectSuccess("Unable to connect to session on: " + coordinationNodePath); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + return session; + } + + private void reconnect() { + // TODO: check id on reconnect + CoordinationSession coordinationSession = connectedSession(); + coordinationSession.describeSemaphore( + semaphoreName, + DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS + ).thenAccept(result -> { + if (!result.isSuccess()) { + logger.error("Unable to describe semaphore {}", semaphoreName); + return; + } + SemaphoreDescription semaphoreDescription = result.getValue(); + SemaphoreDescription.Session owner = semaphoreDescription.getOwnersList().stream().findFirst().get(); + if (owner.getId() != coordinationSession.getId()) { + logger.warn( + "Current session with id: {} lost lease after reconnection on semaphore: {}", + owner.getId(), + semaphoreName + ); + release(); + } + }); + } + + // TODO: interruptible? + public synchronized boolean release() { + logger.debug("Trying to release"); + if (leaseData == null) { + logger.debug("Already released"); + return false; + } + + try { + return leaseData.getProcessLease().release().thenApply(it -> { + logger.debug("Released lock"); + leaseData = null; + return true; + }).get(); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + /** + * @param deadline + * @return true - if successfully acquired lock + * @throws Exception + * @throws LockAlreadyAcquiredException + * @throws LockAcquireFailedException + */ + // TODO: deadlock? + public synchronized LeaseData tryAcquire( + @Nullable Instant deadline, + boolean exclusive, + byte[] data + ) throws Exception { + logger.debug("Trying to acquire with deadline: {}, exclusive: {}", deadline, exclusive); + + if (leaseData != null) { + if (leaseData.isExclusive() == exclusive) { + throw new LockAlreadyAcquiredException( + coordinationNodePath, + semaphoreName + ); + } + if (!leaseData.isExclusive() && exclusive) { + throw new LockUpgradeFailedException( + coordinationNodePath, + semaphoreName + ); + } + } + + Optional lease = tryBlockingLock(deadline, exclusive, data); + if (lease.isPresent()) { + leaseData = new LeaseData(lease.get(), exclusive); + logger.debug("Successfully acquired lock: {}", semaphoreName); + return leaseData; + } + logger.debug("Unable to acquire lock: {}", semaphoreName); + return null; + } + + private Optional tryBlockingLock( + @Nullable Instant deadline, + boolean exclusive, + byte[] data + ) throws Exception { + int retryCount = 0; + CoordinationSession coordinationSession = connectedSession(); + + while (coordinationSession.getState().isActive() && (deadline == null || Instant.now().isBefore(deadline))) { + retryCount++; + + Duration timeout; + if (deadline == null) { + timeout = DEFAULT_TIMEOUT; + } else { + timeout = Duration.between(Instant.now(), deadline); // TODO: use external Clock instead of Instant? + } + + CompletableFuture> acquireTask = coordinationSession.acquireEphemeralSemaphore( + semaphoreName, exclusive, data, timeout // TODO: change Session API to use deadlines + ); + Result leaseResult; + try { + leaseResult = acquireTask.get(); + } catch (InterruptedException e) { + // If acquire is interrupted, then release immediately + Thread.currentThread().interrupt(); + acquireTask.thenAccept(acquireResult -> { + if (!acquireResult.getStatus().isSuccess()) { + return; + } + SemaphoreLease lease = acquireResult.getValue(); + lease.release(); + }); + throw e; + } + + Status status = leaseResult.getStatus(); + logger.debug("Lease result status: {}", status); + + if (status.isSuccess()) { + logger.debug("Successfully acquired the lock"); + return Optional.of(leaseResult.getValue()); + } + + if (status.getCode() == StatusCode.TIMEOUT) { + logger.debug("Trying to acquire semaphore {} again, retries: {}", semaphoreName, retryCount); + continue; + } + + if (!status.getCode().isRetryable(true)) { + status.expectSuccess("Unable to retry acquiring semaphore"); + throw new LockAcquireFailedException(coordinationNodePath, semaphoreName); + } + } + + if (deadline != null && Instant.now().compareTo(deadline) >= 0) { + return Optional.empty(); + } + + throw new LockAcquireFailedException(coordinationNodePath, semaphoreName); + } + + public String getCoordinationNodePath() { + return coordinationNodePath; + } + + public String getSemaphoreName() { + return semaphoreName; + } + + public CoordinationSession getCoordinationSession() { + return session; + } + + public CoordinationSession getConnectedCoordinationSession() { + return connectedSession(); + } + + public @Nullable LeaseData getLeaseData() { + return leaseData; + } + + public boolean isAcquired() { + return leaseData != null; + } + + @Override + public Listenable getListenable() { + return sessionListenerWrapper; + } + + @Override + public void close() { + try { + release(); + } catch (Exception ignored) { + } + + session.close(); + sessionListenerWrapper.clearListeners(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java new file mode 100644 index 000000000..7a268cd9f --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java @@ -0,0 +1,12 @@ +package tech.ydb.coordination.recipes.locks; + +public class LockUpgradeFailedException extends LockAcquireFailedException { + public LockUpgradeFailedException(String coordinationNodePath, String semaphoreName) { + super( + "Unable to upgrade lease from inclusive to exclusive, " + + "name=" + semaphoreName + ", " + "path=" + coordinationNodePath, + coordinationNodePath, + semaphoreName + ); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java new file mode 100644 index 000000000..056288757 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java @@ -0,0 +1,118 @@ +package tech.ydb.coordination.recipes.locks; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableProvider; + +import java.io.Closeable; +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; + +public class ReadWriteInterProcessLock implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(ReadWriteInterProcessLock.class); + + private final LockInternals lockInternals; + private final InternalLock readLock; + private final InternalLock writeLock; + + public ReadWriteInterProcessLock( + CoordinationClient client, + String coordinationNodePath, + String lockName + ) { + this.lockInternals = new LockInternals( + client, coordinationNodePath, lockName + ); + lockInternals.start(); + + this.readLock = new InternalLock(lockInternals, false); + this.writeLock = new InternalLock(lockInternals, true); + } + + public InterProcessLock writeLock() { + return writeLock; + } + + public InterProcessLock readLock() { + // TODO: Если сделали acquire для read lock, когда уже есть write lock? Сейчас игнорим + return readLock; + } + + private static class InternalLock implements InterProcessLock, ListenableProvider { + private final LockInternals lockInternals; + private final boolean isExclusive; + + private InternalLock(LockInternals lockInternals, boolean isExclusive) { + this.lockInternals = lockInternals; + this.isExclusive = isExclusive; + } + + @Override + public void acquire() throws Exception { + if (!isExclusive && isAcquired(true)) { + logger.debug("Write lock acquired, skipping for read lock"); + return; + } + + lockInternals.tryAcquire( + null, + isExclusive, + null + ); + } + + @Override + public boolean acquire(Duration waitDuration) throws Exception { + Objects.requireNonNull(waitDuration, "wait duration must not be null"); + + if (!isExclusive && isAcquired(true)) { + logger.debug("Write lock acquired, skipping for read lock"); + return true; + } + + Instant deadline = Instant.now().plus(waitDuration); + return lockInternals.tryAcquire( + deadline, + isExclusive, + null + ) != null; + } + + @Override + public boolean release() { + if (!isAcquiredInThisProcess()) { + return false; + } + + return lockInternals.release(); + } + + @Override + public boolean isAcquiredInThisProcess() { + return isAcquired(isExclusive); + } + + private boolean isAcquired(boolean exclusive) { + LockInternals.LeaseData leaseData = lockInternals.getLeaseData(); + if (leaseData == null) { + return false; + } + return leaseData.isExclusive() == exclusive; + } + + @Override + public Listenable getListenable() { + return lockInternals.getListenable(); + } + } + + @Override + public void close() { + lockInternals.close(); + } + +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java new file mode 100644 index 000000000..1454de09c --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java @@ -0,0 +1,7 @@ +package tech.ydb.coordination.recipes.util; + +/** + * Marker interface + */ +public interface ByteSerializable { +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java new file mode 100644 index 000000000..736e6400e --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java @@ -0,0 +1,45 @@ +package tech.ydb.coordination.recipes.util; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import java.util.function.Function; + +public interface Listenable { + void addListener(Consumer listener); + + /** + * Listener call will be processed in executor + * @param listener + * @param executor + */ + void addListener(Consumer listener, ExecutorService executor); + + void removeListener(Consumer listener); + + void clearListeners(); + + default CompletableFuture waitUntil(Function condition) { + CompletableFuture future = new CompletableFuture<>(); + Consumer listener = new Consumer() { + @Override + public void accept(T data) { + if (future.isDone()) return; + + if (Thread.currentThread().isInterrupted()) { + future.completeExceptionally(new InterruptedException()); + return; + } + + if (condition.apply(data)) { + future.complete(data); + } + } + }; + + future.whenComplete((result, ex) -> removeListener(listener)); + addListener(listener); + + return future; + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java new file mode 100644 index 000000000..777efbd8b --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java @@ -0,0 +1,8 @@ +package tech.ydb.coordination.recipes.util; + +import java.util.function.Consumer; + +public interface ListenableAdder { + void addListener(Consumer listener); + void removeListener(Consumer listener); +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java new file mode 100644 index 000000000..883915178 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java @@ -0,0 +1,29 @@ +package tech.ydb.coordination.recipes.util; + + +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +public interface ListenableProvider extends Listenable { + Listenable getListenable(); + + @Override + default void addListener(Consumer listener) { + getListenable().addListener(listener); + } + + @Override + default void addListener(Consumer listener, ExecutorService executor) { + getListenable().addListener(listener, executor); + } + + @Override + default void removeListener(Consumer listener) { + getListenable().removeListener(listener); + } + + @Override + default void clearListeners() { + getListenable().clearListeners(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java new file mode 100644 index 000000000..d8e9054eb --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java @@ -0,0 +1,51 @@ +package tech.ydb.coordination.recipes.util; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +public class ListenerWrapper implements Listenable { + private final ListenableAdder listenableAdder; + /** + * key - original (external) consumer, value - consumer wrapper or original consumer depending on executor + */ + private final Map, Consumer> listenersMapping = new HashMap<>(); + + public ListenerWrapper(ListenableAdder listenableAdder) { + this.listenableAdder = listenableAdder; + } + + @Override + public void addListener(Consumer listener) { + if (listenersMapping.containsKey(listener)) { + return; + } + + listenersMapping.put(listener, listener); + listenableAdder.addListener(listener); + } + + @Override + public void addListener(Consumer listener, ExecutorService executor) { + if (listenersMapping.containsKey(listener)) { + return; + } + + Consumer wrapper = T -> executor.submit(() -> listener.accept(T)); + listenersMapping.put(listener, wrapper); + listenableAdder.addListener(wrapper); + } + + @Override + public void removeListener(Consumer listener) { + Consumer removed = listenersMapping.remove(listener); + listenableAdder.removeListener(removed); + } + + @Override + public void clearListeners() { + listenersMapping.keySet().forEach(this::removeListener); + listenersMapping.clear(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java new file mode 100644 index 000000000..7fbc72a39 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java @@ -0,0 +1,10 @@ +package tech.ydb.coordination.recipes.util; + +/** + * Exception thrown during serialization/deserialization. + */ +public class SerializationException extends RuntimeException { + SerializationException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java new file mode 100644 index 000000000..cc8750c03 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java @@ -0,0 +1,18 @@ +package tech.ydb.coordination.recipes.util; + +/** + * Serializes and deserializes objects of type T. + * + * @param the type of object to be serialized/deserialized + */ +public interface Serializer { + /** + * Serializes an object to a byte array. + */ + byte[] serialize(T obj) throws SerializationException; + + /** + * Deserializes a byte array to an object. + */ + T deserialize(byte[] data) throws SerializationException; +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java new file mode 100644 index 000000000..13878d77b --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java @@ -0,0 +1,54 @@ +package tech.ydb.coordination.recipes.util; + +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.CoordinationSession.State; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +public class SessionListenerWrapper implements Listenable { + private final CoordinationSession session; + /** + * key - original (external) consumer, value - consumer wrapper or original consumer depending on executor + */ + private final Map, Consumer> listenersMapping = new HashMap<>(); + + public SessionListenerWrapper(CoordinationSession session) { + this.session = session; + } + + @Override + public void addListener(Consumer listener) { + if (listenersMapping.containsKey(listener)) { + return; + } + + listenersMapping.put(listener, listener); + session.addStateListener(listener); + } + + @Override + public void addListener(Consumer listener, ExecutorService executor) { + if (listenersMapping.containsKey(listener)) { + return; + } + + Consumer wrapper = state -> executor.submit(() -> listener.accept(state)); + listenersMapping.put(listener, wrapper); + session.addStateListener(wrapper); + } + + @Override + public void removeListener(Consumer listener) { + Consumer removed = listenersMapping.remove(listener); + session.removeStateListener(removed); + } + + @Override + public void clearListeners() { + listenersMapping.keySet().forEach(this::removeListener); + listenersMapping.clear(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java new file mode 100644 index 000000000..f3c59fa64 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java @@ -0,0 +1,61 @@ +package tech.ydb.coordination.recipes.watch; + +import java.util.Arrays; +import java.util.Objects; + +public class Participant { + private final long id; + private final byte[] data; + private final long count; + private final boolean isLeader; + + public Participant(long id, byte[] data, long count, boolean isLeader) { + this.id = id; + this.data = data; + this.count = count; + this.isLeader = isLeader; + } + + public long getId() { + return id; + } + + public byte[] getData() { + return data; + } + + public long getCount() { + return count; + } + + public boolean isLeader() { + return isLeader; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Participant that = (Participant) o; + return id == that.id && count == that.count && isLeader == that.isLeader && Objects.deepEquals(data, that.data); + } + + @Override + public int hashCode() { + return Objects.hash(id, Arrays.hashCode(data), count, isLeader); + } + + @Override + public String toString() { + return "Participant{" + + "id=" + id + + ", data=" + Arrays.toString(data) + + ", count=" + count + + ", isLeader=" + isLeader + + '}'; + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java new file mode 100644 index 000000000..e8f48f1eb --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java @@ -0,0 +1,195 @@ +package tech.ydb.coordination.recipes.watch; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableAdder; +import tech.ydb.coordination.recipes.util.ListenableProvider; +import tech.ydb.coordination.recipes.util.ListenerWrapper; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.settings.WatchSemaphoreMode; +import tech.ydb.core.Result; +import tech.ydb.core.Status; + +import java.io.Closeable; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +public class SemaphoreWatchListener implements ListenableAdder, ListenableProvider, Closeable { + private static final Logger logger = LoggerFactory.getLogger(SemaphoreWatchListener.class); + + private final CoordinationSession session; + private final String semaphoreName; + private final ListenerWrapper listenableWrapper; + + private AtomicReference state; + private Future watchTask; + private volatile WatchData watchData; + private Set> listeners; + + public SemaphoreWatchListener(CoordinationSession session, String semaphoreName) { + this.session = session; + this.semaphoreName = semaphoreName; + this.state = new AtomicReference<>(State.CREATED); + this.watchTask = null; + this.watchData = null; + this.listeners = new HashSet<>(); + this.listenableWrapper = new ListenerWrapper<>(this); + } + + public enum State { + CREATED, + STARTED, + CLOSED + } + + public List getOwners() { + // TODO: block until initialized or throw exception or return default value or return Optional.empty() + Preconditions.checkState(watchData == null, "Is not yet fetched state"); + + return Collections.unmodifiableList(watchData.owners); // TODO: copy Participant.data[]? + } + + public List getWaiters() { + Preconditions.checkState(watchData == null, "Is not yet fetched state"); + + return Collections.unmodifiableList(watchData.waiters); // TODO: copy Participant.data[]? + } + + public List getParticipants() { + Preconditions.checkState(watchData == null, "Is not yet fetched state"); + + return Collections.unmodifiableList(watchData.participants); // TODO: copy Participant.data[]? + } + + public long getCount() { + Preconditions.checkState(watchData == null, "Is not yet fetched state"); + + return watchData.count; + } + + public byte[] getData() { + Preconditions.checkState(watchData == null, "Is not yet fetched state"); + + return watchData.data.clone(); + } + + public boolean start() { + Preconditions.checkState(state.compareAndSet(State.CREATED, State.STARTED), "Already started or closed"); + + return enqueueWatch(); + } + + private synchronized boolean enqueueWatch() { + if (watchIsQueued() && state.get() == State.STARTED) { + return false; + } + + watchTask = watchSemaphore().thenCompose(status -> { + if (!status.isSuccess()) { + // TODO: stop watching on error? + logger.error("Wailed to watch semaphore: {} with status: {}", semaphoreName, status); + } + + finish(); + return null; + }); + return true; + } + + private boolean watchIsQueued() { + return watchTask != null; + } + + private synchronized void finish() { + watchTask = null; + enqueueWatch(); + } + + private CompletableFuture watchSemaphore() { + return session.watchSemaphore( + semaphoreName, + DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, + WatchSemaphoreMode.WATCH_DATA_AND_OWNERS + ).thenCompose(result -> { + Status status = result.getStatus(); + if (!status.isSuccess()) { + return CompletableFuture.completedFuture(status); + } + tech.ydb.coordination.description.SemaphoreWatcher watcher = result.getValue(); + saveWatchState(watcher.getDescription()); + return watcher.getChangedFuture().thenApply(Result::getStatus); + }); + } + + private void saveWatchState(SemaphoreDescription description) { + List waitersList = description.getWaitersList().stream().map(it -> new Participant( + it.getId(), + it.getData(), + it.getCount(), + false + )).collect(Collectors.toList()); + List ownersList = description.getOwnersList().stream().map(it -> new Participant( + it.getId(), + it.getData(), + it.getCount(), + true + )).collect(Collectors.toList()); + + watchData = new WatchData( + description.getCount(), + description.getData(), + waitersList, + ownersList + ); + notifyListeners(); + } + + private void notifyListeners() { + listeners.forEach(listener -> listener.accept(watchData)); + } + + private synchronized void stopWatch() { + Future task = watchTask; + if (task != null) { + task.cancel(true); + } + watchTask = null; + } + + public State getState() { + return state.get(); + } + + @Override + public Listenable getListenable() { + return listenableWrapper; + } + + @Override + public void addListener(Consumer listener) { + listeners.add(listener); + } + + @Override + public void removeListener(Consumer listener) { + listeners.remove(listener); + } + + @Override + public void close() { + Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Is not yet started"); + + stopWatch(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java new file mode 100644 index 000000000..ed8a612de --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java @@ -0,0 +1,41 @@ +package tech.ydb.coordination.recipes.watch; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class WatchData { + final long count; + final byte[] data; + final List waiters; + final List owners; + final List participants; + + WatchData(long count, byte[] data, List waiters, List owners) { + this.count = count; + this.data = data; + this.waiters = waiters; + this.owners = owners; + this.participants = Stream.concat(owners.stream(), waiters.stream()).collect(Collectors.toList()); + } + + public long getCount() { + return count; + } + + public byte[] getData() { + return data; + } + + public List getWaiters() { + return waiters; + } + + public List getOwners() { + return owners; + } + + public List getParticipants() { + return participants; + } +} diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java new file mode 100644 index 000000000..7c83a4cd7 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java @@ -0,0 +1,171 @@ +package tech.ydb.coordination; + +import org.junit.Assert; +import org.junit.Before; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.mockito.stubbing.OngoingStubbing; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import tech.ydb.core.Status; +import tech.ydb.core.StatusCode; +import tech.ydb.core.grpc.GrpcReadWriteStream; +import tech.ydb.core.grpc.GrpcTransport; +import tech.ydb.proto.StatusCodesProtos; +import tech.ydb.proto.coordination.SessionRequest; +import tech.ydb.proto.coordination.SessionResponse; +import tech.ydb.proto.coordination.v1.CoordinationServiceGrpc; + +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; + +public class CoordinationSessionBaseMockedTest { + private static final Logger logger = LoggerFactory.getLogger(CoordinationSessionBaseMockedTest.class); + + private final ScheduledExecutorService scheduler = Mockito.mock(ScheduledExecutorService.class); + private final GrpcTransport transport = Mockito.mock(GrpcTransport.class); + private final ScheduledFuture emptyFuture = Mockito.mock(ScheduledFuture.class); + private final GrpcReadWriteStream writeStream = + Mockito.mock(GrpcReadWriteStream.class); + private final SchedulerAssert schedulerHelper = new SchedulerAssert(); + + protected final CoordinationClient client = CoordinationClient.newClient(transport); + + private volatile MockedWriteStream streamMock = null; + + @Before + public void beforeEach() { + Mockito.when(transport.getScheduler()).thenReturn(scheduler); + Mockito.when(transport.readWriteStreamCall(Mockito.eq(CoordinationServiceGrpc.getSessionMethod()), Mockito.any())) + .thenReturn(writeStream); // create mocked stream + + // Every writeStream.start updates mockedWriteStream + Mockito.when(writeStream.start(Mockito.any())) + .thenAnswer(defaultStreamMockAnswer()); + + // Every writeStream.sendNext add message from client to mockedWriteStream.sent list + Mockito.doAnswer((Answer) (InvocationOnMock iom) -> { + streamMock.sent.add(iom.getArgument(0, SessionRequest.class)); + return null; + }).when(writeStream).sendNext(Mockito.any()); + + Mockito.when(scheduler.schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any())) + .thenAnswer((InvocationOnMock iom) -> { + logger.debug("mock scheduled task"); + schedulerHelper.tasks.add(iom.getArgument(0, Runnable.class)); + return emptyFuture; + }); + } + + protected MockedWriteStream currentStream() { + return streamMock; + } + + protected SchedulerAssert getScheduler() { + return schedulerHelper; + } + + protected OngoingStubbing> mockStreams() { + return Mockito.when(writeStream.start(Mockito.any())); + } + + protected Answer> errorStreamMockAnswer(StatusCode code) { + return (iom) -> { + streamMock = null; + return CompletableFuture.completedFuture(Status.of(code)); + }; + } + + protected Answer> defaultStreamMockAnswer() { + return (InvocationOnMock iom) -> { + streamMock = new MockedWriteStream(iom.getArgument(0)); + return streamMock.streamFuture; + }; + } + + protected static class SchedulerAssert { + private final Queue tasks = new ConcurrentLinkedQueue<>(); + + public SchedulerAssert hasNoTasks() { + Assert.assertTrue(tasks.isEmpty()); + return this; + } + + public SchedulerAssert hasTasks(int count) { + Assert.assertEquals(count, tasks.size()); + return this; + } + + public SchedulerAssert executeNextTasks(int count) { + Assert.assertTrue(count <= tasks.size()); + + CompletableFuture.runAsync(() -> { + logger.debug("execute {} scheduled tasks", count); + for (int idx = 0; idx < count; idx++) { + tasks.poll().run(); + } + }).join(); + return this; + } + } + + protected class MockedWriteStream { + private final GrpcReadWriteStream.Observer observer; + private final CompletableFuture streamFuture = new CompletableFuture<>(); + private final List sent = new ArrayList<>(); + private volatile int sentIdx = 0; + + public MockedWriteStream(GrpcReadWriteStream.Observer observer) { + this.observer = observer; + } + + public void complete(Status status) { + streamFuture.complete(status); + } + + public void complete(Throwable th) { + streamFuture.completeExceptionally(th); + } + + public void hasNoNewMessages() { + Assert.assertTrue(sentIdx >= sent.size()); + } + + public Checker nextMsg() { + Assert.assertTrue(sentIdx < sent.size()); + return new Checker(sent.get(sentIdx++)); + } + + public void responseSemaphoreAlreadyExists() { + SessionResponse msg = SessionResponse.newBuilder() + .setAcquireSemaphoreResult( + SessionResponse.AcquireSemaphoreResult.newBuilder().setStatus( + StatusCodesProtos.StatusIds.StatusCode.ALREADY_EXISTS + ) + ) + .build(); + observer.onNext(msg); + } + + } + + protected class Checker { + private final SessionRequest msg; + + public Checker(SessionRequest msg) { + this.msg = msg; + } + + public Checker isAcquireSemaphore() { + Assert.assertTrue("next msg must be acquire semaphore", msg.hasAcquireSemaphore()); + return this; + } + } +} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java new file mode 100644 index 000000000..bf5431b86 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java @@ -0,0 +1,25 @@ +package tech.ydb.coordination.recipes; + +import org.junit.Assert; +import org.junit.Test; + +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.CoordinationSessionBaseMockedTest; +import tech.ydb.core.StatusCode; + +public class ReadWriteLockTest extends CoordinationSessionBaseMockedTest { + + @Test + public void simpleTest() { + mockStreams() + .then(errorStreamMockAnswer(StatusCode.TRANSPORT_UNAVAILABLE)) + .then(defaultStreamMockAnswer()); // and repeat; + + CoordinationSession session = client.createSession("/coordination/node/path"); + session.connect(); + + Assert.assertNull(currentStream()); + getScheduler().hasTasks(1).executeNextTasks(1); + } + +} From 9278dc55c69a8021fb7ac99c50f4917b35f68d4f Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Thu, 24 Apr 2025 01:12:33 +0300 Subject: [PATCH 02/10] added coordination module mocks --- .../CoordinationSessionBaseMockedTest.java | 136 ++++++++++++------ .../recipes/ReadWriteLockTest.java | 47 +++++- 2 files changed, 134 insertions(+), 49 deletions(-) diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java index 7c83a4cd7..466636d71 100644 --- a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java @@ -1,11 +1,11 @@ package tech.ydb.coordination; +import org.jetbrains.annotations.NotNull; import org.junit.Assert; import org.junit.Before; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import org.mockito.stubbing.OngoingStubbing; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -21,10 +21,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.*; public class CoordinationSessionBaseMockedTest { private static final Logger logger = LoggerFactory.getLogger(CoordinationSessionBaseMockedTest.class); @@ -32,29 +29,13 @@ public class CoordinationSessionBaseMockedTest { private final ScheduledExecutorService scheduler = Mockito.mock(ScheduledExecutorService.class); private final GrpcTransport transport = Mockito.mock(GrpcTransport.class); private final ScheduledFuture emptyFuture = Mockito.mock(ScheduledFuture.class); - private final GrpcReadWriteStream writeStream = - Mockito.mock(GrpcReadWriteStream.class); private final SchedulerAssert schedulerHelper = new SchedulerAssert(); protected final CoordinationClient client = CoordinationClient.newClient(transport); - private volatile MockedWriteStream streamMock = null; - @Before public void beforeEach() { Mockito.when(transport.getScheduler()).thenReturn(scheduler); - Mockito.when(transport.readWriteStreamCall(Mockito.eq(CoordinationServiceGrpc.getSessionMethod()), Mockito.any())) - .thenReturn(writeStream); // create mocked stream - - // Every writeStream.start updates mockedWriteStream - Mockito.when(writeStream.start(Mockito.any())) - .thenAnswer(defaultStreamMockAnswer()); - - // Every writeStream.sendNext add message from client to mockedWriteStream.sent list - Mockito.doAnswer((Answer) (InvocationOnMock iom) -> { - streamMock.sent.add(iom.getArgument(0, SessionRequest.class)); - return null; - }).when(writeStream).sendNext(Mockito.any()); Mockito.when(scheduler.schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any())) .thenAnswer((InvocationOnMock iom) -> { @@ -64,35 +45,41 @@ public void beforeEach() { }); } - protected MockedWriteStream currentStream() { - return streamMock; - } - protected SchedulerAssert getScheduler() { return schedulerHelper; } - protected OngoingStubbing> mockStreams() { - return Mockito.when(writeStream.start(Mockito.any())); - } + protected StreamMock mockStream() { + StreamMock streamMock = new StreamMock(); - protected Answer> errorStreamMockAnswer(StatusCode code) { - return (iom) -> { - streamMock = null; - return CompletableFuture.completedFuture(Status.of(code)); - }; - } + GrpcReadWriteStream readWriteStream = Mockito.mock(GrpcReadWriteStream.class); + + Mockito.when(readWriteStream.start(Mockito.any())).thenAnswer( + (InvocationOnMock iom) -> { + streamMock.setObserver(iom.getArgument(0)); + return streamMock.streamFuture; + } + ).thenThrow(new RuntimeException("Unexpected second start call")); - protected Answer> defaultStreamMockAnswer() { - return (InvocationOnMock iom) -> { - streamMock = new MockedWriteStream(iom.getArgument(0)); - return streamMock.streamFuture; - }; + Mockito.doAnswer((Answer) (InvocationOnMock iom) -> { + streamMock.sent.add(iom.getArgument(0, SessionRequest.class)); + return null; + }).when(readWriteStream).sendNext(Mockito.any()); + + Mockito.when(transport.readWriteStreamCall(Mockito.eq(CoordinationServiceGrpc.getSessionMethod()), Mockito.any())) + .thenReturn(readWriteStream); + return streamMock; } - protected static class SchedulerAssert { + protected static class SchedulerAssert implements Executor { private final Queue tasks = new ConcurrentLinkedQueue<>(); + @Override + public void execute(@NotNull Runnable command) { + logger.debug("scheduling command: " + command); + tasks.add(command); + } + public SchedulerAssert hasNoTasks() { Assert.assertTrue(tasks.isEmpty()); return this; @@ -116,16 +103,25 @@ public SchedulerAssert executeNextTasks(int count) { } } - protected class MockedWriteStream { - private final GrpcReadWriteStream.Observer observer; - private final CompletableFuture streamFuture = new CompletableFuture<>(); + protected class StreamMock { + private final CompletableFuture streamFuture; private final List sent = new ArrayList<>(); private volatile int sentIdx = 0; - public MockedWriteStream(GrpcReadWriteStream.Observer observer) { + private volatile GrpcReadWriteStream.Observer observer = null; + + public StreamMock() { + streamFuture = new CompletableFuture<>(); + } + + public void setObserver(GrpcReadWriteStream.Observer observer) { this.observer = observer; } + public void complete(StatusCode statusCode) { + streamFuture.complete(Status.of(statusCode)); + } + public void complete(Status status) { streamFuture.complete(status); } @@ -151,6 +147,34 @@ public void responseSemaphoreAlreadyExists() { ) ) .build(); + response(msg); + } + + public void responseSessionStarted(long sessionId) { + SessionResponse msg = SessionResponse.newBuilder() + .setSessionStarted( + SessionResponse.SessionStarted.newBuilder() + .setSessionId(sessionId) + .build() + ) + .build(); + response(msg); + } + + public void responseAcquiredSuccessfully(long requestId) { + SessionResponse msg = SessionResponse.newBuilder() + .setAcquireSemaphoreResult( + SessionResponse.AcquireSemaphoreResult.newBuilder() + .setReqId(requestId) + .setAcquired(true) + .setStatus(StatusCodesProtos.StatusIds.StatusCode.SUCCESS) + ) + .build(); + response(msg); + } + + private void response(SessionResponse msg) { + Assert.assertNotNull(observer); observer.onNext(msg); } @@ -163,9 +187,33 @@ public Checker(SessionRequest msg) { this.msg = msg; } + public SessionRequest get() { + return msg; + } + public Checker isAcquireSemaphore() { Assert.assertTrue("next msg must be acquire semaphore", msg.hasAcquireSemaphore()); return this; } + + public Checker isEphemeralSemaphore() { + Assert.assertTrue("next msg must be acquire ephemeral semaphore", msg.getAcquireSemaphore().getEphemeral()); + return this; + } + + public Checker hasSemaphoreName(String semaphoreName) { + Assert.assertEquals("invalid semaphore name", semaphoreName, msg.getAcquireSemaphore().getName()); + return this; + } + + public Checker isSessionStart() { + Assert.assertTrue("next msg must be session start", msg.hasSessionStart()); + return this; + } + + public Checker hasPath(String coordinationNodePath) { + Assert.assertEquals("invalid coordination node path", coordinationNodePath, msg.getSessionStart().getPath()); + return this; + } } } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java index bf5431b86..cc3637c28 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java @@ -5,21 +5,58 @@ import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.CoordinationSessionBaseMockedTest; +import tech.ydb.coordination.SemaphoreLease; +import tech.ydb.coordination.settings.CoordinationSessionSettings; +import tech.ydb.core.Result; import tech.ydb.core.StatusCode; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; + public class ReadWriteLockTest extends CoordinationSessionBaseMockedTest { @Test - public void simpleTest() { - mockStreams() - .then(errorStreamMockAnswer(StatusCode.TRANSPORT_UNAVAILABLE)) - .then(defaultStreamMockAnswer()); // and repeat; + public void startTest() throws Exception { + StreamMock streamMock = mockStream(); + streamMock.complete(StatusCode.TRANSPORT_UNAVAILABLE); CoordinationSession session = client.createSession("/coordination/node/path"); session.connect(); + streamMock.nextMsg().isSessionStart().hasPath("/coordination/node/path"); + + Assert.assertEquals(CoordinationSession.State.INITIAL, session.getState()); + } + + @Test + public void successAcquireTest() throws Exception { + StreamMock streamMock = mockStream(); + + CoordinationSession session = client.createSession( + "/coordination/node/path", + CoordinationSessionSettings.newBuilder() + .withExecutor(getScheduler()) + .build() + ); + session.connect(); + streamMock.nextMsg().isSessionStart().hasPath("/coordination/node/path"); + streamMock.responseSessionStarted(123); - Assert.assertNull(currentStream()); getScheduler().hasTasks(1).executeNextTasks(1); + Assert.assertEquals(CoordinationSession.State.CONNECTED, session.getState()); + + CompletableFuture> leaseResult = session.acquireEphemeralSemaphore("lock", false, Duration.ofSeconds(1)); + long requestId = streamMock.nextMsg().isAcquireSemaphore() + .isEphemeralSemaphore() + .hasSemaphoreName("lock") + .get() + .getAcquireSemaphore() + .getReqId(); + streamMock.responseAcquiredSuccessfully(requestId); + getScheduler().hasTasks(1).executeNextTasks(1); + Result lease = leaseResult.join(); + + Assert.assertTrue(lease.isSuccess()); + Assert.assertEquals("lock", lease.getValue().getSemaphoreName()); } } From 53bacfaa324bdc751aab72c9b26537f120f9e655 Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Mon, 5 May 2025 17:48:44 +0300 Subject: [PATCH 03/10] feat: added group membership --- .../ydb/coordination/impl/SessionImpl.java | 2 +- .../ElectionParticipant.java} | 32 +- .../recipes/election/LeaderElection.java | 276 +++++++++ .../recipes/election/LeaderElector.java | 199 ------ .../recipes/group/GroupMember.java | 47 ++ .../recipes/group/GroupMembership.java | 26 + .../recipes/group/GroupMembershipImpl.java | 124 ++++ .../recipes/locks/InterProcessLock.java | 9 +- .../recipes/locks/InterProcessMutex.java | 30 +- .../locks/InterProcessMutexSettings.java | 31 + .../recipes/locks/LockInternals.java | 229 ++++--- .../locks/LockReleaseFailedException.java | 38 ++ .../locks/LockUpgradeFailedException.java | 12 - .../locks/ReadWriteInterProcessLock.java | 70 +-- .../recipes/util/ByteSerializable.java | 7 - .../coordination/recipes/util/Listenable.java | 28 - .../recipes/util/ListenableAdder.java | 8 - .../recipes/util/ListenableContainer.java | 54 ++ .../recipes/util/ListenableProvider.java | 29 - .../recipes/util/ListenerWrapper.java | 51 -- .../recipes/util/SemaphoreObserver.java | 139 +++++ .../recipes/util/SerializationException.java | 10 - .../coordination/recipes/util/Serializer.java | 18 - .../util/SessionListenableProvider.java | 7 + .../recipes/util/SessionListenerWrapper.java | 54 -- .../recipes/watch/SemaphoreWatchListener.java | 195 ------ .../coordination/recipes/watch/WatchData.java | 41 -- .../CoordinationSessionBaseMockedTest.java | 333 +++++----- .../CoordinationStreamBaseMockedTest.java | 223 +++++++ .../recipes/ReadWriteLockTest.java | 62 -- .../LeaderElectionIntegrationTest.java | 185 ++++++ .../group/GroupMembershipIntegrationTest.java | 66 ++ .../InterProcessMutexIntegrationTest.java | 218 +++++++ .../recipes/locks/InternalLockMockedTest.java | 572 ++++++++++++++++++ ...dWriteInterProcessLockIntegrationTest.java | 302 +++++++++ .../util/SemaphoreObserverMockedTest.java | 29 + 36 files changed, 2726 insertions(+), 1030 deletions(-) rename coordination/src/main/java/tech/ydb/coordination/recipes/{watch/Participant.java => election/ElectionParticipant.java} (52%) create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMember.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java delete mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java diff --git a/coordination/src/main/java/tech/ydb/coordination/impl/SessionImpl.java b/coordination/src/main/java/tech/ydb/coordination/impl/SessionImpl.java index a9ae47cec..3a7eadefb 100644 --- a/coordination/src/main/java/tech/ydb/coordination/impl/SessionImpl.java +++ b/coordination/src/main/java/tech/ydb/coordination/impl/SessionImpl.java @@ -162,7 +162,7 @@ private CompletableFuture> connectToSession(Stream stream, long ses } }, executor); - // and send session start message with id of previos session (or zero if it's first connect) + // and send session start message with id of previous session (or zero if it's first connect) return stream.sendSessionStart(sessionID, nodePath, connectTimeout, protectionKey); } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/ElectionParticipant.java similarity index 52% rename from coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java rename to coordination/src/main/java/tech/ydb/coordination/recipes/election/ElectionParticipant.java index f3c59fa64..b05c48616 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/Participant.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/ElectionParticipant.java @@ -1,33 +1,27 @@ -package tech.ydb.coordination.recipes.watch; +package tech.ydb.coordination.recipes.election; import java.util.Arrays; import java.util.Objects; -public class Participant { - private final long id; +public class ElectionParticipant { + private final long sessionId; private final byte[] data; - private final long count; private final boolean isLeader; - public Participant(long id, byte[] data, long count, boolean isLeader) { - this.id = id; + public ElectionParticipant(long id, byte[] data, boolean isLeader) { + this.sessionId = id; this.data = data; - this.count = count; this.isLeader = isLeader; } - public long getId() { - return id; + public long getSessionId() { + return sessionId; } public byte[] getData() { return data; } - public long getCount() { - return count; - } - public boolean isLeader() { return isLeader; } @@ -40,21 +34,21 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - Participant that = (Participant) o; - return id == that.id && count == that.count && isLeader == that.isLeader && Objects.deepEquals(data, that.data); + ElectionParticipant that = (ElectionParticipant) o; + return sessionId == that.sessionId && isLeader == that.isLeader && + Objects.deepEquals(data, that.data); } @Override public int hashCode() { - return Objects.hash(id, Arrays.hashCode(data), count, isLeader); + return Objects.hash(sessionId, Arrays.hashCode(data), isLeader); } @Override public String toString() { - return "Participant{" + - "id=" + id + + return "ElectionParticipant{" + + "sessionId=" + sessionId + ", data=" + Arrays.toString(data) + - ", count=" + count + ", isLeader=" + isLeader + '}'; } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java new file mode 100644 index 000000000..d98e5418f --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -0,0 +1,276 @@ +package tech.ydb.coordination.recipes.election; + +import java.io.Closeable; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import tech.ydb.common.retry.RetryForever; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.recipes.locks.LockInternals; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.SessionListenableProvider; +import tech.ydb.coordination.recipes.util.SemaphoreObserver; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.settings.WatchSemaphoreMode; +import tech.ydb.core.StatusCode; + +// TODO: настройки + переименовать переменные + рекомендации по коду + логгирование + backoff политика +public class LeaderElection implements Closeable, SessionListenableProvider { + private static final Logger logger = LoggerFactory.getLogger(LeaderElection.class); + private static final long MAX_LEASE = 1L; + + private final CoordinationClient client; + private final LeaderElectionListener leaderElectionListener; + private final String coordinationNodePath; + private final String electionName; + private final byte[] data; + private final ExecutorService electionExecutor; + private final LockInternals lock; + private final SemaphoreObserver semaphoreObserver; + + private AtomicReference state = new AtomicReference<>(State.CREATED); + private volatile boolean autoRequeue = false; + private volatile boolean isLeader = false; + private Future electionTask = null; + + private enum State { + CREATED, + STARTED, + CLOSED + } + + public LeaderElection( + CoordinationClient client, + LeaderElectionListener leaderElectionListener, + String coordinationNodePath, + String electionName, + byte[] data + ) { + this( + client, + leaderElectionListener, + coordinationNodePath, + electionName, + data, + Executors.newSingleThreadExecutor() + ); + } + + public LeaderElection( + CoordinationClient client, + LeaderElectionListener leaderElectionListener, + String coordinationNodePath, + String electionName, + byte[] data, + ExecutorService executorService + ) { + this.client = client; + this.leaderElectionListener = leaderElectionListener; + this.coordinationNodePath = coordinationNodePath; + this.electionName = electionName; + this.data = data; + this.electionExecutor = executorService; + this.lock = new LockInternals( + MAX_LEASE, + client, + coordinationNodePath, + electionName + ); + this.semaphoreObserver = new SemaphoreObserver( + lock.getCoordinationSession(), + electionName, + WatchSemaphoreMode.WATCH_OWNERS, + DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, + new RetryForever(100) // TODO: передавать снаружи + ); + } + + public void start() { + Preconditions.checkState(state.compareAndSet(State.CREATED, State.STARTED), "Already started or closed"); + // TODO: create session? + CoordinationSession coordinationSession = lock.getCoordinationSession(); + // TODO: retry on create? Non idempotent - will not be retried automatically + lock.start(); + coordinationSession.createSemaphore(electionName, MAX_LEASE).thenAccept(status -> { + if (status.isSuccess() || status.getCode() == StatusCode.ALREADY_EXISTS) { + semaphoreObserver.start(); + } + status.expectSuccess("Unable to create semaphore"); + // TODO: set status == error + }); + + if (autoRequeue) { + enqueueElection(); + } + } + + public void autoRequeue() { + autoRequeue = true; + } + + public boolean isLeader() { + return isLeader; + } + + /** + * Re-queue an attempt for leadership. If this instance is already queued, nothing + * happens and false is returned. If the instance was not queued, it is re-queued and true + * is returned + * + * @return true if re-enqueue was successful + */ + public boolean requeue() { + Preconditions.checkState(state.get() == State.STARTED, "Already closed or not yet started"); + + // TODO: корректно обрабатывать если старт еще не кончился + return enqueueElection(); + } + + public synchronized boolean interruptLeadership() { + Future localTask = electionTask; + if (localTask != null) { + localTask.cancel(true); + electionTask = null; + return true; + } + return false; + } + + private synchronized boolean enqueueElection() { + if (!isQueued() && state.get() == State.STARTED) { + electionTask = electionExecutor.submit(new Callable() { + @Override + public Void call() throws Exception { + try { + doWork(); + } finally { + finishTask(); + } + return null; + } + }); + return true; + } + + return false; + } + + private void doWork() throws Exception { + isLeader = false; + + try { + lock.getConnectedCoordinationSession(); // asserts that session is connected or throws exception + lock.tryAcquire( + null, + true, + data + ); + isLeader = true; + try { + leaderElectionListener.takeLeadership(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw e; + } catch (Throwable e) { + logger.debug("takeLeadership exception", e); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw e; + } finally { + if (isLeader) { + isLeader = false; + boolean wasInterrupted = Thread.interrupted(); + try { + lock.release(); + } catch (Exception e) { + logger.error("Lock release exception for: " + coordinationNodePath); + } finally { + if (wasInterrupted) { + Thread.currentThread().interrupt(); + } + } + } + } + } + + private synchronized void finishTask() { + electionTask = null; + if (autoRequeue) { + enqueueElection(); + } + } + + private boolean isQueued() { + return electionTask != null; + } + + /** + * Не гарантированы все, кроме лидера + * @return + */ + public List getParticipants() { + SemaphoreDescription semaphoreDescription = semaphoreObserver.getCachedData(); + if (semaphoreDescription == null) { + return Collections.emptyList(); + } + + return Stream.concat( + semaphoreDescription.getOwnersList().stream() + .map(session -> mapParticipant(session, true)), + semaphoreDescription.getWaitersList().stream() + .map(session -> mapParticipant(session, false)) + ).collect(Collectors.toList()); + } + + public Optional getCurrentLeader() { + SemaphoreDescription semaphoreDescription = semaphoreObserver.getCachedData(); + if (semaphoreDescription == null) { + return Optional.empty(); + } + + return semaphoreDescription.getOwnersList().stream().findFirst() + .map(session -> mapParticipant(session, true)); + } + + private static ElectionParticipant mapParticipant(SemaphoreDescription.Session session, boolean owner) { + return new ElectionParticipant( + session.getId(), + session.getData(), + owner + ); + } + + @Override + public Listenable getSessionListenable() { + return lock.getSessionListenable(); + } + + @Override + public synchronized void close() { + Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Already closed"); + + Future localTask = electionTask; + if (localTask != null) { + localTask.cancel(true); + electionTask = null; + } + + electionExecutor.shutdown(); + semaphoreObserver.close(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java deleted file mode 100644 index c4e65d82f..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElector.java +++ /dev/null @@ -1,199 +0,0 @@ -package tech.ydb.coordination.recipes.election; - -import com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import tech.ydb.coordination.CoordinationClient; -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.recipes.locks.LockInternals; -import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.ListenableProvider; -import tech.ydb.coordination.recipes.watch.Participant; -import tech.ydb.coordination.recipes.watch.SemaphoreWatchListener; - -import java.io.Closeable; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; - -public class LeaderElector implements Closeable, ListenableProvider { - private static final Logger logger = LoggerFactory.getLogger(LeaderElector.class); - - private final CoordinationClient client; - private final LeaderElectionListener leaderElectionListener; - private final String coordinationNodePath; - private final String semaphoreName; - private final ExecutorService electionExecutor; - private final LockInternals lock; - private final SemaphoreWatchListener semaphoreWatchAdapter; - - private AtomicReference state = new AtomicReference<>(State.STARTED); - private volatile boolean autoRequeue = false; - private volatile boolean isLeader = false; - private Future electionTask = null; - - - private enum State { // TODO: needs third state (CREATED)? - STARTED, - CLOSED - } - - public LeaderElector( - CoordinationClient client, - LeaderElectionListener leaderElectionListener, - String coordinationNodePath, - String semaphoreName - ) { - this(client, leaderElectionListener, coordinationNodePath, semaphoreName, Executors.newSingleThreadExecutor()); - } - - public LeaderElector( - CoordinationClient client, - LeaderElectionListener leaderElectionListener, - String coordinationNodePath, - String semaphoreName, - ExecutorService executorService - ) { - this.client = client; - this.leaderElectionListener = leaderElectionListener; - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = semaphoreName; - this.electionExecutor = executorService; - this.lock = new LockInternals( - client, - coordinationNodePath, - semaphoreName - ); - this.lock.start(); - this.semaphoreWatchAdapter = new SemaphoreWatchListener(lock.getCoordinationSession(), semaphoreName); - semaphoreWatchAdapter.start(); - } - - public boolean isLeader() { - return isLeader; - } - - public synchronized void interruptLeadership() { - Future task = electionTask; - if (task != null) { - task.cancel(true); - } - } - - /** - * Re-queue an attempt for leadership. If this instance is already queued, nothing - * happens and false is returned. If the instance was not queued, it is re-queued and true - * is returned - * - * @return true if re-enqueue was successful - */ - public boolean requeue() { - Preconditions.checkState(state.get() == State.STARTED, "Already closed or not yet started"); - - return enqueueElection(); - } - - public void autoRequeue() { - autoRequeue = true; - } - - private synchronized boolean enqueueElection() { - if (!isQueued() && state.get() == State.STARTED) { - electionTask = electionExecutor.submit(new Callable() { - @Override - public Void call() throws Exception { - try { - doWork(); - } finally { - finishTask(); - } - return null; - } - }); - return true; - } - - return false; - } - - private void doWork() throws Exception { - isLeader = false; - - try { - lock.tryAcquire( - null, - true, - null - ); - isLeader = true; - try { - leaderElectionListener.takeLeadership(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw e; - } catch (Throwable e) { - logger.debug("takeLeadership exception", e); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw e; - } finally { - if (isLeader) { - isLeader = false; - boolean wasInterrupted = Thread.interrupted(); - try { - lock.release(); - } catch (Exception e) { - logger.error("Lock release exception for: " + coordinationNodePath); - } finally { - if (wasInterrupted) { - Thread.currentThread().interrupt(); - } - } - } - } - } - - private synchronized void finishTask() { - electionTask = null; - if (autoRequeue) { // TODO: requeue if critical exception? - enqueueElection(); - } - } - - private boolean isQueued() { - return electionTask != null; - } - - public List getParticipants() { - return semaphoreWatchAdapter.getParticipants(); - } - - public Optional getLeader() { - return semaphoreWatchAdapter.getOwners().stream().findFirst(); - } - - @Override - public synchronized void close() { - Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Already closed"); - - Future task = electionTask; - if (task != null) { - task.cancel(true); - } - - electionTask = null; - electionExecutor.shutdown(); - semaphoreWatchAdapter.close(); - getListenable().clearListeners(); - } - - @Override - public Listenable getListenable() { - return lock.getListenable(); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMember.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMember.java new file mode 100644 index 000000000..9a7cbbd93 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMember.java @@ -0,0 +1,47 @@ +package tech.ydb.coordination.recipes.group; + +import java.util.Arrays; +import java.util.Objects; + +public class GroupMember { + private final long sessionId; + private final byte[] data; + + public GroupMember(long sessionId, byte[] data) { + this.sessionId = sessionId; + this.data = data; + } + + public long getSessionId() { + return sessionId; + } + + public byte[] getData() { + return data; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + GroupMember that = (GroupMember) o; + return sessionId == that.sessionId && Objects.deepEquals(data, that.data); + } + + @Override + public int hashCode() { + return Objects.hash(sessionId, Arrays.hashCode(data)); + } + + @Override + public String toString() { + return "GroupMember{" + + "sessionId=" + sessionId + + ", data=" + Arrays.toString(data) + + '}'; + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java new file mode 100644 index 000000000..7fc4e9ccc --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java @@ -0,0 +1,26 @@ +package tech.ydb.coordination.recipes.group; + +import java.io.Closeable; +import java.util.List; + +import org.checkerframework.checker.nullness.qual.Nullable; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.SessionListenableProvider; + +public interface GroupMembership extends Closeable, SessionListenableProvider { + /** + * Adds instance to the group and start observing members + */ + void start(); + + /** + * Get cached members of the group or null + */ + @Nullable + List getCurrentMembers(); + + /** + * Get listenable to subscribe to members list update + */ + Listenable> getMembersListenable(); +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java new file mode 100644 index 000000000..cb17b9942 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java @@ -0,0 +1,124 @@ +package tech.ydb.coordination.recipes.group; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +import org.checkerframework.checker.nullness.qual.Nullable; +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.recipes.locks.LockInternals; +import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableContainer; +import tech.ydb.coordination.recipes.util.SemaphoreObserver; +import tech.ydb.coordination.settings.CoordinationSessionSettings; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.settings.WatchSemaphoreMode; + +public class GroupMembershipImpl implements GroupMembership { + private static final long MAX_GROUP_SIZE = Long.MAX_VALUE; + + private final CoordinationClient coordinationClient; + private final String coordinationNodePath; + private final String groupName; + private final RetryPolicy retryPolicy; + + private final CoordinationSession session; + private final LockInternals lockInternals; + private final SemaphoreObserver semaphoreObserver; + private final ListenableContainer sessionStateListenable; + private final ListenableContainer> groupMembersListenable; + + public GroupMembershipImpl( + CoordinationClient coordinationClient, + String coordinationNodePath, + String groupName, + RetryPolicy retryPolicy + ) { + this.coordinationClient = coordinationClient; + this.coordinationNodePath = coordinationNodePath; + this.groupName = groupName; + this.retryPolicy = retryPolicy; + + this.session = coordinationClient.createSession( + coordinationNodePath, + CoordinationSessionSettings.newBuilder() + .withRetryPolicy(retryPolicy) + .build() + ); + this.sessionStateListenable = new ListenableContainer<>(); + session.addStateListener(sessionStateListenable::notifyListeners); + + this.lockInternals = new LockInternals( + session, + groupName, + MAX_GROUP_SIZE + ); + + this.semaphoreObserver = new SemaphoreObserver( + session, + groupName, + WatchSemaphoreMode.WATCH_OWNERS, + DescribeSemaphoreMode.WITH_OWNERS, + retryPolicy + ); + this.groupMembersListenable = new ListenableContainer<>(); + semaphoreObserver.getWatchDataListenable().addListener(description -> { + List groupMembers = mapSemaphoreDescriptionToMembersList(description); + groupMembersListenable.notifyListeners(groupMembers); + }); + } + + @Override + public void start() { + // TODO: correctly handle failed connection, failed semaphore and failed starts + session.connect().thenAccept(sessionStatus -> { + sessionStatus.expectSuccess("Unable to establish session"); + session.createSemaphore(groupName, MAX_GROUP_SIZE).thenAccept(semaphoreStatus -> { + lockInternals.start(); + semaphoreObserver.start(); + }); + }); + } + + @Override + public @Nullable List getCurrentMembers() { + SemaphoreDescription cachedDescription = semaphoreObserver.getCachedData(); + return mapSemaphoreDescriptionToMembersList(cachedDescription); + } + + private static @Nullable List mapSemaphoreDescriptionToMembersList(SemaphoreDescription description) { + if (description == null) { + return null; + } + + List ownersList = description.getOwnersList(); + return ownersList.stream().map(GroupMembershipImpl::mapSessionToGroupMember).collect(Collectors.toList()); + } + + private static GroupMember mapSessionToGroupMember(SemaphoreDescription.Session session) { + return new GroupMember( + session.getId(), + session.getData() + ); + } + + @Override + public Listenable getSessionListenable() { + return sessionStateListenable; + } + + @Override + public Listenable> getMembersListenable() { + return groupMembersListenable; + } + + @Override + public void close() throws IOException { + session.close(); + lockInternals.close(); + semaphoreObserver.close(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java index 4f66b5a35..e1b7737b0 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java @@ -1,11 +1,10 @@ package tech.ydb.coordination.recipes.locks; -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.recipes.util.Listenable; - import java.time.Duration; -public interface InterProcessLock extends Listenable { +import tech.ydb.coordination.recipes.util.SessionListenableProvider; + +public interface InterProcessLock extends SessionListenableProvider { void acquire() throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException; /** @@ -16,7 +15,7 @@ public interface InterProcessLock extends Listenable /** * @return false if nothing to release */ - boolean release() throws Exception; + boolean release() throws InterruptedException, LockReleaseFailedException; /** * @return true if the lock is acquired by a thread in this JVM diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java index 6a3fc6883..2167d2529 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java @@ -1,26 +1,28 @@ package tech.ydb.coordination.recipes.locks; -import tech.ydb.coordination.CoordinationClient; -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.ListenableProvider; - -import javax.annotation.concurrent.ThreadSafe; import java.io.Closeable; import java.time.Duration; import java.time.Instant; -@ThreadSafe -public class InterProcessMutex implements InterProcessLock, ListenableProvider, Closeable { +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.util.Listenable; + +// TODO: add documentation and logs +public class InterProcessMutex implements InterProcessLock, Closeable { private final LockInternals lockInternals; public InterProcessMutex( CoordinationClient client, String coordinationNodePath, - String lockName + String lockName, + InterProcessMutexSettings settings ) { lockInternals = new LockInternals(client, coordinationNodePath, lockName); lockInternals.start(); + if (settings.isWaitConnection()) { + lockInternals.getConnectedCoordinationSession(); + } } @Override @@ -43,7 +45,7 @@ public boolean acquire(Duration waitDuration) throws Exception { } @Override - public boolean release() throws Exception { + public boolean release() throws InterruptedException { return lockInternals.release(); } @@ -53,12 +55,12 @@ public boolean isAcquiredInThisProcess() { } @Override - public Listenable getListenable() { - return null; + public void close() { + lockInternals.close(); } @Override - public void close() { - lockInternals.close(); + public Listenable getSessionListenable() { + return lockInternals.getSessionListenable(); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java new file mode 100644 index 000000000..5de77842c --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java @@ -0,0 +1,31 @@ +package tech.ydb.coordination.recipes.locks; + +// TODO: More settings +public class InterProcessMutexSettings { + private final boolean waitConnection; + + public InterProcessMutexSettings(Builder builder) { + this.waitConnection = builder.waitConnection; + } + + public boolean isWaitConnection() { + return waitConnection; + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private boolean waitConnection = false; + + public Builder withWaitConnection(boolean waitConnection) { + this.waitConnection = waitConnection; + return this; + } + + public InterProcessMutexSettings build() { + return new InterProcessMutexSettings(this); + } + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java index 10749120a..b66e060c5 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java @@ -1,77 +1,110 @@ package tech.ydb.coordination.recipes.locks; +import java.io.Closeable; +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; + import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.SemaphoreLease; -import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.recipes.util.ListenableContainer; import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.ListenableProvider; -import tech.ydb.coordination.recipes.util.SessionListenerWrapper; -import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.recipes.util.SessionListenableProvider; import tech.ydb.core.Result; import tech.ydb.core.Status; import tech.ydb.core.StatusCode; -import javax.annotation.concurrent.ThreadSafe; -import java.io.Closeable; -import java.time.Duration; -import java.time.Instant; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.function.Consumer; - -@ThreadSafe -public class LockInternals implements ListenableProvider, Closeable { +public class LockInternals implements Closeable, SessionListenableProvider { private static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(30); + private static final Logger logger = LoggerFactory.getLogger(LockInternals.class); + private final boolean persistent; + private final long maxPersistentLease; private final String coordinationNodePath; private final String semaphoreName; private final CoordinationSession session; - private final SessionListenerWrapper sessionListenerWrapper; + private final ListenableContainer sessionListenable = new ListenableContainer(); private CompletableFuture sessionConnectionTask = null; - private volatile LeaseData leaseData = null; // TODO: needs to be volatile? + private volatile LeaseData leaseData = null; public static class LeaseData { private final SemaphoreLease processLease; - private final boolean isExclusive; + private final boolean exclusive; + private final long leaseSessionId; - public LeaseData(SemaphoreLease processLease, boolean isExclusive) { + public LeaseData(SemaphoreLease processLease, boolean exclusive, long leaseSessionId) { this.processLease = processLease; - this.isExclusive = isExclusive; + this.exclusive = exclusive; + this.leaseSessionId = leaseSessionId; } public boolean isExclusive() { - return isExclusive; + return exclusive; } public SemaphoreLease getProcessLease() { return processLease; } + public long getLeaseSessionId() { + return leaseSessionId; + } + @Override public String toString() { return "LeaseData{" + "processLease=" + processLease + - ", isExclusive=" + isExclusive + + ", isExclusive=" + exclusive + + ", leaseSessionId=" + leaseSessionId + '}'; } } public LockInternals( + CoordinationSession session, + String lockName, + long maxPersistentLease + ) { + this.persistent = false; + this.maxPersistentLease = -1; + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = lockName; + this.session = client.createSession(coordinationNodePath); + } + + public LockInternals( + CoordinationClient client, + String coordinationNodePath, + String lockName + ) { + this.persistent = false; + this.maxPersistentLease = -1; + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = lockName; + this.session = client.createSession(coordinationNodePath); + } + + public LockInternals( + long maxPersistentLease, CoordinationClient client, String coordinationNodePath, String lockName ) { + this.persistent = true; + this.maxPersistentLease = maxPersistentLease; this.coordinationNodePath = coordinationNodePath; this.semaphoreName = lockName; this.session = client.createSession(coordinationNodePath); - this.sessionListenerWrapper = new SessionListenerWrapper(session); } public void start() { @@ -89,15 +122,17 @@ public void start() { } case CLOSED: { logger.debug("Session CLOSED, releasing lock"); - release(); + leaseData = null; break; } case LOST: { logger.debug("Session LOST, releasing lock"); - release(); + leaseData = null; break; } + default: } + sessionListenable.notifyListeners(state); }; session.addStateListener(listener); @@ -116,86 +151,75 @@ private CoordinationSession connectedSession() { } private void reconnect() { - // TODO: check id on reconnect - CoordinationSession coordinationSession = connectedSession(); - coordinationSession.describeSemaphore( - semaphoreName, - DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS - ).thenAccept(result -> { - if (!result.isSuccess()) { - logger.error("Unable to describe semaphore {}", semaphoreName); - return; - } - SemaphoreDescription semaphoreDescription = result.getValue(); - SemaphoreDescription.Session owner = semaphoreDescription.getOwnersList().stream().findFirst().get(); - if (owner.getId() != coordinationSession.getId()) { - logger.warn( - "Current session with id: {} lost lease after reconnection on semaphore: {}", - owner.getId(), - semaphoreName - ); - release(); - } - }); + LeaseData currentLeaseData = leaseData; + long oldId = currentLeaseData.getLeaseSessionId(); + long newId = session.getId(); + if (oldId != newId) { + logger.warn( + "Current session with new id: {} lost lease after reconnection on semaphore: {}", + newId, + semaphoreName + ); + leaseData = null; + } else { + logger.debug("Successfully reestablished session with same id: {}", newId); + } } - // TODO: interruptible? - public synchronized boolean release() { - logger.debug("Trying to release"); - if (leaseData == null) { - logger.debug("Already released"); + public synchronized boolean release() throws LockReleaseFailedException, InterruptedException { + logger.debug("Trying to release semaphore '{}'", semaphoreName); + + if (!connectedSession().getState().isActive()) { + throw new LockReleaseFailedException( + "Coordination session is inactive", + coordinationNodePath, + semaphoreName + ); + } + + LeaseData localLeaseData = leaseData; + if (localLeaseData == null) { + logger.debug("Semaphore '{}' already released", semaphoreName); return false; } try { - return leaseData.getProcessLease().release().thenApply(it -> { - logger.debug("Released lock"); - leaseData = null; - return true; - }).get(); - } catch (ExecutionException e) { - throw new RuntimeException(e); + localLeaseData.getProcessLease().release().get(); + leaseData = null; + logger.debug("Successfully released semaphore '{}'", semaphoreName); + return true; } catch (InterruptedException e) { - throw new RuntimeException(e); + Thread.currentThread().interrupt(); + throw e; + } catch (ExecutionException e) { + throw new LockReleaseFailedException( + "Failed to release lock: " + e.getCause().getMessage(), + coordinationNodePath, + semaphoreName, + e.getCause() + ); } } - /** - * @param deadline - * @return true - if successfully acquired lock - * @throws Exception - * @throws LockAlreadyAcquiredException - * @throws LockAcquireFailedException - */ - // TODO: deadlock? public synchronized LeaseData tryAcquire( @Nullable Instant deadline, boolean exclusive, - byte[] data + byte @Nullable [] data ) throws Exception { - logger.debug("Trying to acquire with deadline: {}, exclusive: {}", deadline, exclusive); + logger.debug("Trying to acquire lock: {} with deadline: {}, exclusive: {}", semaphoreName, deadline, exclusive); if (leaseData != null) { - if (leaseData.isExclusive() == exclusive) { - throw new LockAlreadyAcquiredException( - coordinationNodePath, - semaphoreName - ); - } - if (!leaseData.isExclusive() && exclusive) { - throw new LockUpgradeFailedException( - coordinationNodePath, - semaphoreName - ); - } + throw new LockAlreadyAcquiredException(coordinationNodePath, semaphoreName); } Optional lease = tryBlockingLock(deadline, exclusive, data); if (lease.isPresent()) { - leaseData = new LeaseData(lease.get(), exclusive); + LeaseData localLeaseData = new LeaseData(lease.get(), exclusive, 1); + leaseData = localLeaseData; logger.debug("Successfully acquired lock: {}", semaphoreName); - return leaseData; + return localLeaseData; } + logger.debug("Unable to acquire lock: {}", semaphoreName); return null; } @@ -203,7 +227,7 @@ public synchronized LeaseData tryAcquire( private Optional tryBlockingLock( @Nullable Instant deadline, boolean exclusive, - byte[] data + byte @Nullable [] data ) throws Exception { int retryCount = 0; CoordinationSession coordinationSession = connectedSession(); @@ -215,12 +239,13 @@ private Optional tryBlockingLock( if (deadline == null) { timeout = DEFAULT_TIMEOUT; } else { - timeout = Duration.between(Instant.now(), deadline); // TODO: use external Clock instead of Instant? + timeout = Duration.between(Instant.now(), deadline); } - CompletableFuture> acquireTask = coordinationSession.acquireEphemeralSemaphore( - semaphoreName, exclusive, data, timeout // TODO: change Session API to use deadlines + CompletableFuture> acquireTask = acquire( + exclusive, data, coordinationSession, timeout ); + Result leaseResult; try { leaseResult = acquireTask.get(); @@ -241,7 +266,7 @@ private Optional tryBlockingLock( logger.debug("Lease result status: {}", status); if (status.isSuccess()) { - logger.debug("Successfully acquired the lock"); + logger.debug("Successfully acquired the lock '{}'", semaphoreName); return Optional.of(leaseResult.getValue()); } @@ -251,7 +276,7 @@ private Optional tryBlockingLock( } if (!status.getCode().isRetryable(true)) { - status.expectSuccess("Unable to retry acquiring semaphore"); + logger.debug("Unable to retry acquiring semaphore '{}'", semaphoreName); throw new LockAcquireFailedException(coordinationNodePath, semaphoreName); } } @@ -263,6 +288,23 @@ private Optional tryBlockingLock( throw new LockAcquireFailedException(coordinationNodePath, semaphoreName); } + private CompletableFuture> acquire( + boolean exclusive, + byte[] data, + CoordinationSession coordinationSession, + Duration timeout + ) { + if (!persistent) { + return coordinationSession.acquireEphemeralSemaphore(semaphoreName, exclusive, data, timeout); + } + + if (exclusive) { + return coordinationSession.acquireSemaphore(semaphoreName, maxPersistentLease, data, timeout); + } + + return coordinationSession.acquireSemaphore(semaphoreName, 1, data, timeout); + } + public String getCoordinationNodePath() { return coordinationNodePath; } @@ -287,9 +329,13 @@ public boolean isAcquired() { return leaseData != null; } + public boolean isPersistent() { + return persistent; + } + @Override - public Listenable getListenable() { - return sessionListenerWrapper; + public Listenable getSessionListenable() { + return sessionListenable; } @Override @@ -300,6 +346,5 @@ public void close() { } session.close(); - sessionListenerWrapper.clearListeners(); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java new file mode 100644 index 000000000..413fc6bbd --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java @@ -0,0 +1,38 @@ +package tech.ydb.coordination.recipes.locks; + +public class LockReleaseFailedException extends RuntimeException { + private final String coordinationNodePath; + private final String semaphoreName; + + public LockReleaseFailedException(String message, String coordinationNodePath, String semaphoreName, Throwable e) { + super( + "Failed to release semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath + + ": '" + message + "'", e + ); + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = semaphoreName; + } + + public LockReleaseFailedException(String message, String coordinationNodePath, String semaphoreName) { + super( + "Failed to release semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath + + ": '" + message + "'" + ); + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = semaphoreName; + } + + public LockReleaseFailedException(String coordinationNodePath, String semaphoreName) { + super("Failed to release semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath); + this.coordinationNodePath = coordinationNodePath; + this.semaphoreName = semaphoreName; + } + + public String getCoordinationNodePath() { + return coordinationNodePath; + } + + public String getSemaphoreName() { + return semaphoreName; + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java deleted file mode 100644 index 7a268cd9f..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockUpgradeFailedException.java +++ /dev/null @@ -1,12 +0,0 @@ -package tech.ydb.coordination.recipes.locks; - -public class LockUpgradeFailedException extends LockAcquireFailedException { - public LockUpgradeFailedException(String coordinationNodePath, String semaphoreName) { - super( - "Unable to upgrade lease from inclusive to exclusive, " + - "name=" + semaphoreName + ", " + "path=" + coordinationNodePath, - coordinationNodePath, - semaphoreName - ); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java index 056288757..d81e7edf9 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java @@ -1,21 +1,23 @@ package tech.ydb.coordination.recipes.locks; +import java.io.Closeable; +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.ListenableProvider; - -import java.io.Closeable; -import java.time.Duration; -import java.time.Instant; -import java.util.Objects; +// TODO: add documentation and logs public class ReadWriteInterProcessLock implements Closeable { private static final Logger logger = LoggerFactory.getLogger(ReadWriteInterProcessLock.class); - private final LockInternals lockInternals; + private final LockInternals readLockInternals; + private final LockInternals writeLockInternals; private final InternalLock readLock; private final InternalLock writeLock; @@ -24,13 +26,21 @@ public ReadWriteInterProcessLock( String coordinationNodePath, String lockName ) { - this.lockInternals = new LockInternals( - client, coordinationNodePath, lockName + this.readLockInternals = new LockInternals( + client, + coordinationNodePath, + lockName ); - lockInternals.start(); + readLockInternals.start(); + this.readLock = new InternalLock(readLockInternals, false); - this.readLock = new InternalLock(lockInternals, false); - this.writeLock = new InternalLock(lockInternals, true); + this.writeLockInternals = new LockInternals( + client, + coordinationNodePath, + lockName + ); + writeLockInternals.start(); + this.writeLock = new InternalLock(writeLockInternals, true); } public InterProcessLock writeLock() { @@ -38,11 +48,10 @@ public InterProcessLock writeLock() { } public InterProcessLock readLock() { - // TODO: Если сделали acquire для read lock, когда уже есть write lock? Сейчас игнорим return readLock; } - private static class InternalLock implements InterProcessLock, ListenableProvider { + private static class InternalLock implements InterProcessLock { private final LockInternals lockInternals; private final boolean isExclusive; @@ -53,11 +62,6 @@ private InternalLock(LockInternals lockInternals, boolean isExclusive) { @Override public void acquire() throws Exception { - if (!isExclusive && isAcquired(true)) { - logger.debug("Write lock acquired, skipping for read lock"); - return; - } - lockInternals.tryAcquire( null, isExclusive, @@ -69,11 +73,6 @@ public void acquire() throws Exception { public boolean acquire(Duration waitDuration) throws Exception { Objects.requireNonNull(waitDuration, "wait duration must not be null"); - if (!isExclusive && isAcquired(true)) { - logger.debug("Write lock acquired, skipping for read lock"); - return true; - } - Instant deadline = Instant.now().plus(waitDuration); return lockInternals.tryAcquire( deadline, @@ -83,36 +82,25 @@ public boolean acquire(Duration waitDuration) throws Exception { } @Override - public boolean release() { - if (!isAcquiredInThisProcess()) { - return false; - } - + public boolean release() throws InterruptedException { return lockInternals.release(); } @Override public boolean isAcquiredInThisProcess() { - return isAcquired(isExclusive); - } - - private boolean isAcquired(boolean exclusive) { - LockInternals.LeaseData leaseData = lockInternals.getLeaseData(); - if (leaseData == null) { - return false; - } - return leaseData.isExclusive() == exclusive; + return lockInternals.isAcquired(); } @Override - public Listenable getListenable() { - return lockInternals.getListenable(); + public Listenable getSessionListenable() { + return lockInternals.getSessionListenable(); } } @Override public void close() { - lockInternals.close(); + readLockInternals.close(); + writeLockInternals.close(); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java deleted file mode 100644 index 1454de09c..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ByteSerializable.java +++ /dev/null @@ -1,7 +0,0 @@ -package tech.ydb.coordination.recipes.util; - -/** - * Marker interface - */ -public interface ByteSerializable { -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java index 736e6400e..6097d0b0b 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java @@ -1,9 +1,7 @@ package tech.ydb.coordination.recipes.util; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.function.Consumer; -import java.util.function.Function; public interface Listenable { void addListener(Consumer listener); @@ -16,30 +14,4 @@ public interface Listenable { void addListener(Consumer listener, ExecutorService executor); void removeListener(Consumer listener); - - void clearListeners(); - - default CompletableFuture waitUntil(Function condition) { - CompletableFuture future = new CompletableFuture<>(); - Consumer listener = new Consumer() { - @Override - public void accept(T data) { - if (future.isDone()) return; - - if (Thread.currentThread().isInterrupted()) { - future.completeExceptionally(new InterruptedException()); - return; - } - - if (condition.apply(data)) { - future.complete(data); - } - } - }; - - future.whenComplete((result, ex) -> removeListener(listener)); - addListener(listener); - - return future; - } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java deleted file mode 100644 index 777efbd8b..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableAdder.java +++ /dev/null @@ -1,8 +0,0 @@ -package tech.ydb.coordination.recipes.util; - -import java.util.function.Consumer; - -public interface ListenableAdder { - void addListener(Consumer listener); - void removeListener(Consumer listener); -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java new file mode 100644 index 000000000..8e1c788f3 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java @@ -0,0 +1,54 @@ +package tech.ydb.coordination.recipes.util; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ListenableContainer implements Listenable { + private static final Logger logger = LoggerFactory.getLogger(ListenableContainer.class); + + private final Map, Consumer> listenersMapping = new ConcurrentHashMap<>(); + + public void notifyListeners(T data) { + listenersMapping.values().forEach(listener -> { + try { + listener.accept(data); + } catch (Exception ex) { + logger.error("Listener threw exception", ex); + } + }); + } + + @Override + public void addListener(Consumer listener) { + if (listenersMapping.containsKey(listener)) { + return; + } + + listenersMapping.put(listener, listener); + } + + @Override + public void addListener(Consumer listener, ExecutorService executor) { + if (listenersMapping.containsKey(listener)) { + return; + } + + Consumer wrapper = new Consumer() { + @Override + public void accept(T data) { + executor.submit(() -> listener.accept(data)); + } + }; + listenersMapping.put(listener, wrapper); + } + + @Override + public void removeListener(Consumer listener) { + listenersMapping.remove(listener); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java deleted file mode 100644 index 883915178..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableProvider.java +++ /dev/null @@ -1,29 +0,0 @@ -package tech.ydb.coordination.recipes.util; - - -import java.util.concurrent.ExecutorService; -import java.util.function.Consumer; - -public interface ListenableProvider extends Listenable { - Listenable getListenable(); - - @Override - default void addListener(Consumer listener) { - getListenable().addListener(listener); - } - - @Override - default void addListener(Consumer listener, ExecutorService executor) { - getListenable().addListener(listener, executor); - } - - @Override - default void removeListener(Consumer listener) { - getListenable().removeListener(listener); - } - - @Override - default void clearListeners() { - getListenable().clearListeners(); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java deleted file mode 100644 index d8e9054eb..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenerWrapper.java +++ /dev/null @@ -1,51 +0,0 @@ -package tech.ydb.coordination.recipes.util; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.function.Consumer; - -public class ListenerWrapper implements Listenable { - private final ListenableAdder listenableAdder; - /** - * key - original (external) consumer, value - consumer wrapper or original consumer depending on executor - */ - private final Map, Consumer> listenersMapping = new HashMap<>(); - - public ListenerWrapper(ListenableAdder listenableAdder) { - this.listenableAdder = listenableAdder; - } - - @Override - public void addListener(Consumer listener) { - if (listenersMapping.containsKey(listener)) { - return; - } - - listenersMapping.put(listener, listener); - listenableAdder.addListener(listener); - } - - @Override - public void addListener(Consumer listener, ExecutorService executor) { - if (listenersMapping.containsKey(listener)) { - return; - } - - Consumer wrapper = T -> executor.submit(() -> listener.accept(T)); - listenersMapping.put(listener, wrapper); - listenableAdder.addListener(wrapper); - } - - @Override - public void removeListener(Consumer listener) { - Consumer removed = listenersMapping.remove(listener); - listenableAdder.removeListener(removed); - } - - @Override - public void clearListeners() { - listenersMapping.keySet().forEach(this::removeListener); - listenersMapping.clear(); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java new file mode 100644 index 000000000..661739d7c --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java @@ -0,0 +1,139 @@ +package tech.ydb.coordination.recipes.util; + +import java.io.Closeable; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; + +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.description.SemaphoreWatcher; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.settings.WatchSemaphoreMode; +import tech.ydb.core.Result; +import tech.ydb.core.Status; + +public class SemaphoreObserver implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(SemaphoreObserver.class); + + private final CoordinationSession session; + private final String semaphoreName; + private final WatchSemaphoreMode watchSemaphoreMode; + private final DescribeSemaphoreMode describeSemaphoreMode; + private final RetryPolicy retryPolicy; + private final ListenableContainer watchDataListenable = new ListenableContainer<>(); + private final ListenableContainer sessionStateListenable = new ListenableContainer<>(); + + private AtomicReference state = new AtomicReference<>(State.CREATED); + private AtomicReference cachedData = new AtomicReference<>(null); + private Future watchTask; + + public SemaphoreObserver( + CoordinationSession session, + String semaphoreName, + WatchSemaphoreMode watchSemaphoreMode, + DescribeSemaphoreMode describeSemaphoreMode, + RetryPolicy retryPolicy + ) { + this.session = Objects.requireNonNull(session, "session cannot be null"); + this.semaphoreName = Objects.requireNonNull(semaphoreName, "semaphoreName cannot be null"); + this.watchSemaphoreMode = watchSemaphoreMode; + this.describeSemaphoreMode = describeSemaphoreMode; + this.retryPolicy = retryPolicy; + + this.session.addStateListener(state -> { + if (!state.isActive()) { + close(); + } + // TODO: clear data after reconnect? + sessionStateListenable.notifyListeners(state); + }); + } + + public enum State { + CREATED, + STARTED, + CLOSED + } + + public void start() { + if (state.compareAndSet(State.CREATED, State.STARTED)) { + // TODO: first describe + enqueueWatch(); + } + } + + private synchronized boolean enqueueWatch() { + if (watchTask != null && state.get() == State.STARTED) { + return false; + } + + watchTask = watchSemaphore().thenCompose(status -> { + if (!status.isSuccess()) { + // TODO: backoff via retryPolicy + logger.error("Failed to watch semaphore: {} with status: {}", semaphoreName, status); + } + + finish(); + return null; + }); + return true; + } + + private synchronized void finish() { + watchTask = null; + if (state.get() == State.STARTED) { + enqueueWatch(); + } + } + + private CompletableFuture watchSemaphore() { + return session.watchSemaphore( + semaphoreName, + describeSemaphoreMode, + watchSemaphoreMode + ).thenCompose(result -> { + Status status = result.getStatus(); + if (!status.isSuccess()) { + return CompletableFuture.completedFuture(status); + } + SemaphoreWatcher watcher = result.getValue(); + saveWatchState(watcher.getDescription()); + return watcher.getChangedFuture().thenApply(Result::getStatus); + }); + } + + private void saveWatchState(SemaphoreDescription description) { + logger.info("Changed semaphore state from {} to {}", cachedData.get(), description); + cachedData.set(description); + watchDataListenable.notifyListeners(description); + } + + private synchronized void stopTaskInternal() { + Future localWatchTask = watchTask; + if (localWatchTask != null) { + localWatchTask.cancel(true); + watchTask = null; + } + } + + public Listenable getWatchDataListenable() { + return watchDataListenable; + } + + public @Nullable SemaphoreDescription getCachedData() { + return cachedData.get(); + } + + @Override + public void close() { + state.set(State.CLOSED); + stopTaskInternal(); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java deleted file mode 100644 index 7fbc72a39..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SerializationException.java +++ /dev/null @@ -1,10 +0,0 @@ -package tech.ydb.coordination.recipes.util; - -/** - * Exception thrown during serialization/deserialization. - */ -public class SerializationException extends RuntimeException { - SerializationException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java deleted file mode 100644 index cc8750c03..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Serializer.java +++ /dev/null @@ -1,18 +0,0 @@ -package tech.ydb.coordination.recipes.util; - -/** - * Serializes and deserializes objects of type T. - * - * @param the type of object to be serialized/deserialized - */ -public interface Serializer { - /** - * Serializes an object to a byte array. - */ - byte[] serialize(T obj) throws SerializationException; - - /** - * Deserializes a byte array to an object. - */ - T deserialize(byte[] data) throws SerializationException; -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java new file mode 100644 index 000000000..5d455ad3e --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java @@ -0,0 +1,7 @@ +package tech.ydb.coordination.recipes.util; + +import tech.ydb.coordination.CoordinationSession; + +public interface SessionListenableProvider { + Listenable getSessionListenable(); +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java deleted file mode 100644 index 13878d77b..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenerWrapper.java +++ /dev/null @@ -1,54 +0,0 @@ -package tech.ydb.coordination.recipes.util; - -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.CoordinationSession.State; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.function.Consumer; - -public class SessionListenerWrapper implements Listenable { - private final CoordinationSession session; - /** - * key - original (external) consumer, value - consumer wrapper or original consumer depending on executor - */ - private final Map, Consumer> listenersMapping = new HashMap<>(); - - public SessionListenerWrapper(CoordinationSession session) { - this.session = session; - } - - @Override - public void addListener(Consumer listener) { - if (listenersMapping.containsKey(listener)) { - return; - } - - listenersMapping.put(listener, listener); - session.addStateListener(listener); - } - - @Override - public void addListener(Consumer listener, ExecutorService executor) { - if (listenersMapping.containsKey(listener)) { - return; - } - - Consumer wrapper = state -> executor.submit(() -> listener.accept(state)); - listenersMapping.put(listener, wrapper); - session.addStateListener(wrapper); - } - - @Override - public void removeListener(Consumer listener) { - Consumer removed = listenersMapping.remove(listener); - session.removeStateListener(removed); - } - - @Override - public void clearListeners() { - listenersMapping.keySet().forEach(this::removeListener); - listenersMapping.clear(); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java deleted file mode 100644 index e8f48f1eb..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/SemaphoreWatchListener.java +++ /dev/null @@ -1,195 +0,0 @@ -package tech.ydb.coordination.recipes.watch; - -import com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.description.SemaphoreDescription; -import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.ListenableAdder; -import tech.ydb.coordination.recipes.util.ListenableProvider; -import tech.ydb.coordination.recipes.util.ListenerWrapper; -import tech.ydb.coordination.settings.DescribeSemaphoreMode; -import tech.ydb.coordination.settings.WatchSemaphoreMode; -import tech.ydb.core.Result; -import tech.ydb.core.Status; - -import java.io.Closeable; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -public class SemaphoreWatchListener implements ListenableAdder, ListenableProvider, Closeable { - private static final Logger logger = LoggerFactory.getLogger(SemaphoreWatchListener.class); - - private final CoordinationSession session; - private final String semaphoreName; - private final ListenerWrapper listenableWrapper; - - private AtomicReference state; - private Future watchTask; - private volatile WatchData watchData; - private Set> listeners; - - public SemaphoreWatchListener(CoordinationSession session, String semaphoreName) { - this.session = session; - this.semaphoreName = semaphoreName; - this.state = new AtomicReference<>(State.CREATED); - this.watchTask = null; - this.watchData = null; - this.listeners = new HashSet<>(); - this.listenableWrapper = new ListenerWrapper<>(this); - } - - public enum State { - CREATED, - STARTED, - CLOSED - } - - public List getOwners() { - // TODO: block until initialized or throw exception or return default value or return Optional.empty() - Preconditions.checkState(watchData == null, "Is not yet fetched state"); - - return Collections.unmodifiableList(watchData.owners); // TODO: copy Participant.data[]? - } - - public List getWaiters() { - Preconditions.checkState(watchData == null, "Is not yet fetched state"); - - return Collections.unmodifiableList(watchData.waiters); // TODO: copy Participant.data[]? - } - - public List getParticipants() { - Preconditions.checkState(watchData == null, "Is not yet fetched state"); - - return Collections.unmodifiableList(watchData.participants); // TODO: copy Participant.data[]? - } - - public long getCount() { - Preconditions.checkState(watchData == null, "Is not yet fetched state"); - - return watchData.count; - } - - public byte[] getData() { - Preconditions.checkState(watchData == null, "Is not yet fetched state"); - - return watchData.data.clone(); - } - - public boolean start() { - Preconditions.checkState(state.compareAndSet(State.CREATED, State.STARTED), "Already started or closed"); - - return enqueueWatch(); - } - - private synchronized boolean enqueueWatch() { - if (watchIsQueued() && state.get() == State.STARTED) { - return false; - } - - watchTask = watchSemaphore().thenCompose(status -> { - if (!status.isSuccess()) { - // TODO: stop watching on error? - logger.error("Wailed to watch semaphore: {} with status: {}", semaphoreName, status); - } - - finish(); - return null; - }); - return true; - } - - private boolean watchIsQueued() { - return watchTask != null; - } - - private synchronized void finish() { - watchTask = null; - enqueueWatch(); - } - - private CompletableFuture watchSemaphore() { - return session.watchSemaphore( - semaphoreName, - DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, - WatchSemaphoreMode.WATCH_DATA_AND_OWNERS - ).thenCompose(result -> { - Status status = result.getStatus(); - if (!status.isSuccess()) { - return CompletableFuture.completedFuture(status); - } - tech.ydb.coordination.description.SemaphoreWatcher watcher = result.getValue(); - saveWatchState(watcher.getDescription()); - return watcher.getChangedFuture().thenApply(Result::getStatus); - }); - } - - private void saveWatchState(SemaphoreDescription description) { - List waitersList = description.getWaitersList().stream().map(it -> new Participant( - it.getId(), - it.getData(), - it.getCount(), - false - )).collect(Collectors.toList()); - List ownersList = description.getOwnersList().stream().map(it -> new Participant( - it.getId(), - it.getData(), - it.getCount(), - true - )).collect(Collectors.toList()); - - watchData = new WatchData( - description.getCount(), - description.getData(), - waitersList, - ownersList - ); - notifyListeners(); - } - - private void notifyListeners() { - listeners.forEach(listener -> listener.accept(watchData)); - } - - private synchronized void stopWatch() { - Future task = watchTask; - if (task != null) { - task.cancel(true); - } - watchTask = null; - } - - public State getState() { - return state.get(); - } - - @Override - public Listenable getListenable() { - return listenableWrapper; - } - - @Override - public void addListener(Consumer listener) { - listeners.add(listener); - } - - @Override - public void removeListener(Consumer listener) { - listeners.remove(listener); - } - - @Override - public void close() { - Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Is not yet started"); - - stopWatch(); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java b/coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java deleted file mode 100644 index ed8a612de..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/watch/WatchData.java +++ /dev/null @@ -1,41 +0,0 @@ -package tech.ydb.coordination.recipes.watch; - -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -public class WatchData { - final long count; - final byte[] data; - final List waiters; - final List owners; - final List participants; - - WatchData(long count, byte[] data, List waiters, List owners) { - this.count = count; - this.data = data; - this.waiters = waiters; - this.owners = owners; - this.participants = Stream.concat(owners.stream(), waiters.stream()).collect(Collectors.toList()); - } - - public long getCount() { - return count; - } - - public byte[] getData() { - return data; - } - - public List getWaiters() { - return waiters; - } - - public List getOwners() { - return owners; - } - - public List getParticipants() { - return participants; - } -} diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java index 466636d71..8aa08ae89 100644 --- a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java @@ -1,219 +1,264 @@ package tech.ydb.coordination; -import org.jetbrains.annotations.NotNull; +import java.time.Duration; +import java.util.HashSet; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.function.Consumer; + import org.junit.Assert; import org.junit.Before; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.mockito.stubbing.OngoingStubbing; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import tech.ydb.core.Result; import tech.ydb.core.Status; import tech.ydb.core.StatusCode; -import tech.ydb.core.grpc.GrpcReadWriteStream; -import tech.ydb.core.grpc.GrpcTransport; -import tech.ydb.proto.StatusCodesProtos; -import tech.ydb.proto.coordination.SessionRequest; -import tech.ydb.proto.coordination.SessionResponse; -import tech.ydb.proto.coordination.v1.CoordinationServiceGrpc; - -import java.util.ArrayList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.*; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.when; public class CoordinationSessionBaseMockedTest { private static final Logger logger = LoggerFactory.getLogger(CoordinationSessionBaseMockedTest.class); - private final ScheduledExecutorService scheduler = Mockito.mock(ScheduledExecutorService.class); - private final GrpcTransport transport = Mockito.mock(GrpcTransport.class); - private final ScheduledFuture emptyFuture = Mockito.mock(ScheduledFuture.class); - private final SchedulerAssert schedulerHelper = new SchedulerAssert(); - - protected final CoordinationClient client = CoordinationClient.newClient(transport); + private final CoordinationClient client = Mockito.mock(CoordinationClient.class); + private final CoordinationSession coordinationSession = Mockito.mock(CoordinationSession.class); + private final SessionMock sessionMock = new SessionMock(); + private final SessionStateAssert sessionStateAssert = new SessionStateAssert(); @Before public void beforeEach() { - Mockito.when(transport.getScheduler()).thenReturn(scheduler); + when(client.createSession(any())).thenReturn(coordinationSession); + + when(coordinationSession.connect()) + .thenReturn( + CompletableFuture.completedFuture( + Status.of(StatusCode.TRANSPORT_UNAVAILABLE) + ) + ); + + doAnswer((InvocationOnMock iom) -> { + Consumer consumer = iom.getArgument(0, Consumer.class); + logger.debug("Add session mock listener={}", consumer); + sessionMock.addListener(consumer); + return null; + }).when(coordinationSession).addStateListener(any()); - Mockito.when(scheduler.schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any())) - .thenAnswer((InvocationOnMock iom) -> { - logger.debug("mock scheduled task"); - schedulerHelper.tasks.add(iom.getArgument(0, Runnable.class)); - return emptyFuture; - }); - } + doAnswer((InvocationOnMock iom) -> { + Consumer consumer = iom.getArgument(0, Consumer.class); + logger.debug("Remove session mock listener={}", consumer); + sessionMock.removeListener(consumer); + return null; + }).when(coordinationSession).removeStateListener(any()); - protected SchedulerAssert getScheduler() { - return schedulerHelper; + doAnswer((InvocationOnMock iom) -> { + logger.debug("Get mock state={}", sessionMock.state); + return sessionMock.state; + }).when(coordinationSession).getState(); } - protected StreamMock mockStream() { - StreamMock streamMock = new StreamMock(); + public CoordinationSession getCoordinationSession() { + return coordinationSession; + } - GrpcReadWriteStream readWriteStream = Mockito.mock(GrpcReadWriteStream.class); + public SessionStateAssert getSessionStateAssert() { + return sessionStateAssert; + } - Mockito.when(readWriteStream.start(Mockito.any())).thenAnswer( - (InvocationOnMock iom) -> { - streamMock.setObserver(iom.getArgument(0)); - return streamMock.streamFuture; - } - ).thenThrow(new RuntimeException("Unexpected second start call")); + protected Answer> successConnect() { + return (InvocationOnMock iom) -> { + logger.debug("Successful session connect"); + return CompletableFuture.completedFuture( + Status.SUCCESS + ); + }; + } - Mockito.doAnswer((Answer) (InvocationOnMock iom) -> { - streamMock.sent.add(iom.getArgument(0, SessionRequest.class)); - return null; - }).when(readWriteStream).sendNext(Mockito.any()); + protected Answer> failedConnect(StatusCode code) { + return (InvocationOnMock iom) -> { + logger.debug("Failed session connect, code={}", code); + return CompletableFuture.completedFuture( + Status.of(code) + ); + }; + } - Mockito.when(transport.readWriteStreamCall(Mockito.eq(CoordinationServiceGrpc.getSessionMethod()), Mockito.any())) - .thenReturn(readWriteStream); - return streamMock; + protected Answer>> successAcquire(SemaphoreLease lease) { + return (InvocationOnMock iom) -> { + logger.debug("Success semaphore acquire {}", lease.getSemaphoreName()); + return CompletableFuture.completedFuture( + Result.success(lease) + ); + }; } - protected static class SchedulerAssert implements Executor { - private final Queue tasks = new ConcurrentLinkedQueue<>(); + protected Answer>> statusAcquire(StatusCode statusCode) { + return (InvocationOnMock iom) -> { + logger.debug("Response semaphore acquire with code: {}", statusCode); + return CompletableFuture.completedFuture( + Result.fail(Status.of(statusCode)) + ); + }; + } - @Override - public void execute(@NotNull Runnable command) { - logger.debug("scheduling command: " + command); - tasks.add(command); - } + protected Answer>> timeoutAcquire() { + return (InvocationOnMock iom) -> { + logger.debug("Timeout semaphore acquire"); + return CompletableFuture.completedFuture( + Result.fail(Status.of(StatusCode.TIMEOUT)) + ); + }; + } - public SchedulerAssert hasNoTasks() { - Assert.assertTrue(tasks.isEmpty()); - return this; - } + protected Answer>> timeoutAcquire(Duration blockDuration) { + return (InvocationOnMock iom) -> { + logger.debug("Block acquire duration={}", blockDuration); + Thread.sleep(blockDuration.toMillis()); + return CompletableFuture.completedFuture( + Result.fail(Status.of(StatusCode.TIMEOUT)) + ); + }; + } - public SchedulerAssert hasTasks(int count) { - Assert.assertEquals(count, tasks.size()); - return this; - } + protected Answer>> lostAcquire() { + return (InvocationOnMock iom) -> { + logger.debug("Lost session during"); + sessionMock.lost(); + return CompletableFuture.completedFuture( + Result.fail(Status.of(StatusCode.TIMEOUT)) + ); + }; + } - public SchedulerAssert executeNextTasks(int count) { - Assert.assertTrue(count <= tasks.size()); + protected LeaseMock lease(String semaphoreName) { + return new LeaseMock( + sessionMock, + semaphoreName, + coordinationSession + ); + } - CompletableFuture.runAsync(() -> { - logger.debug("execute {} scheduled tasks", count); - for (int idx = 0; idx < count; idx++) { - tasks.poll().run(); - } - }).join(); - return this; - } + protected CoordinationClient getClient() { + return client; } - protected class StreamMock { - private final CompletableFuture streamFuture; - private final List sent = new ArrayList<>(); - private volatile int sentIdx = 0; + protected SessionMock getSessionMock() { + return sessionMock; + } - private volatile GrpcReadWriteStream.Observer observer = null; + protected class SessionStateAssert implements Consumer { + private final Queue queue = new ConcurrentLinkedQueue<>(); - public StreamMock() { - streamFuture = new CompletableFuture<>(); + public SessionStateAssert next(CoordinationSession.State state) { + queue.add(state); + return this; } - public void setObserver(GrpcReadWriteStream.Observer observer) { - this.observer = observer; + @Override + public void accept(CoordinationSession.State state) { + logger.debug("Next state: {}", state); + Assert.assertFalse(queue.isEmpty()); + CoordinationSession.State lastState = queue.poll(); + Assert.assertEquals(state, lastState); } - public void complete(StatusCode statusCode) { - streamFuture.complete(Status.of(statusCode)); + public void finished() { + Assert.assertTrue(queue.isEmpty()); } + } - public void complete(Status status) { - streamFuture.complete(status); + protected class LeaseMock implements SemaphoreLease { + private final SessionMock sessionMock; + private final String name; + private final CoordinationSession session; + private boolean released = false; + private CompletableFuture result = CompletableFuture.completedFuture(null); + + public LeaseMock(SessionMock sessionMock, String name, CoordinationSession session) { + this.sessionMock = sessionMock; + this.name = name; + this.session = session; } - public void complete(Throwable th) { - streamFuture.completeExceptionally(th); + @Override + public CoordinationSession getSession() { + return session; } - public void hasNoNewMessages() { - Assert.assertTrue(sentIdx >= sent.size()); + @Override + public String getSemaphoreName() { + return name; } - public Checker nextMsg() { - Assert.assertTrue(sentIdx < sent.size()); - return new Checker(sent.get(sentIdx++)); + @Override + public CompletableFuture release() { + released = true; + return result; } - public void responseSemaphoreAlreadyExists() { - SessionResponse msg = SessionResponse.newBuilder() - .setAcquireSemaphoreResult( - SessionResponse.AcquireSemaphoreResult.newBuilder().setStatus( - StatusCodesProtos.StatusIds.StatusCode.ALREADY_EXISTS - ) - ) - .build(); - response(msg); + public LeaseMock failed(Exception exception) { + result = new CompletableFuture<>(); + result.completeExceptionally(exception); + return this; } - public void responseSessionStarted(long sessionId) { - SessionResponse msg = SessionResponse.newBuilder() - .setSessionStarted( - SessionResponse.SessionStarted.newBuilder() - .setSessionId(sessionId) - .build() - ) - .build(); - response(msg); + public void assertReleased() { + Assert.assertTrue(released); } + } - public void responseAcquiredSuccessfully(long requestId) { - SessionResponse msg = SessionResponse.newBuilder() - .setAcquireSemaphoreResult( - SessionResponse.AcquireSemaphoreResult.newBuilder() - .setReqId(requestId) - .setAcquired(true) - .setStatus(StatusCodesProtos.StatusIds.StatusCode.SUCCESS) - ) - .build(); - response(msg); - } + protected class SessionMock { + private final Set> listeners = new HashSet<>(); + + private CoordinationSession.State state = CoordinationSession.State.INITIAL; - private void response(SessionResponse msg) { - Assert.assertNotNull(observer); - observer.onNext(msg); + public SessionMock() { } - } + private void addListener(Consumer consumer) { + listeners.add(consumer); + } - protected class Checker { - private final SessionRequest msg; + private void removeListener(Consumer consumer) { + listeners.remove(consumer); + } - public Checker(SessionRequest msg) { - this.msg = msg; + public OngoingStubbing> connect() { + return when(coordinationSession.connect()); } - public SessionRequest get() { - return msg; + public OngoingStubbing>> acquireEphemeralSemaphore() { + return when(coordinationSession.acquireEphemeralSemaphore(anyString(), anyBoolean(), any(), any())); } - public Checker isAcquireSemaphore() { - Assert.assertTrue("next msg must be acquire semaphore", msg.hasAcquireSemaphore()); - return this; + public void connecting() { + changeState(CoordinationSession.State.CONNECTING); } - public Checker isEphemeralSemaphore() { - Assert.assertTrue("next msg must be acquire ephemeral semaphore", msg.getAcquireSemaphore().getEphemeral()); - return this; + public void connected() { + changeState(CoordinationSession.State.CONNECTED); } - public Checker hasSemaphoreName(String semaphoreName) { - Assert.assertEquals("invalid semaphore name", semaphoreName, msg.getAcquireSemaphore().getName()); - return this; + public void lost() { + changeState(CoordinationSession.State.LOST); } - public Checker isSessionStart() { - Assert.assertTrue("next msg must be session start", msg.hasSessionStart()); - return this; + public void closed() { + changeState(CoordinationSession.State.CLOSED); } - public Checker hasPath(String coordinationNodePath) { - Assert.assertEquals("invalid coordination node path", coordinationNodePath, msg.getSessionStart().getPath()); - return this; + private void changeState(CoordinationSession.State newState) { + state = newState; + listeners.forEach(it -> it.accept(newState)); } } + } diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java new file mode 100644 index 000000000..e858701c9 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java @@ -0,0 +1,223 @@ +package tech.ydb.coordination; + +import org.jetbrains.annotations.NotNull; +import org.junit.Assert; +import org.junit.Before; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import tech.ydb.core.Status; +import tech.ydb.core.StatusCode; +import tech.ydb.core.grpc.GrpcReadWriteStream; +import tech.ydb.core.grpc.GrpcTransport; +import tech.ydb.proto.StatusCodesProtos; +import tech.ydb.proto.coordination.SessionRequest; +import tech.ydb.proto.coordination.SessionResponse; +import tech.ydb.proto.coordination.v1.CoordinationServiceGrpc; + +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; + +public class CoordinationStreamBaseMockedTest { + private static final Logger logger = LoggerFactory.getLogger(CoordinationStreamBaseMockedTest.class); + + private final ScheduledExecutorService scheduler = Mockito.mock(ScheduledExecutorService.class); + private final GrpcTransport transport = Mockito.mock(GrpcTransport.class); + private final ScheduledFuture emptyFuture = Mockito.mock(ScheduledFuture.class); + private final SchedulerAssert schedulerHelper = new SchedulerAssert(); + + protected final CoordinationClient client = CoordinationClient.newClient(transport); + + @Before + public void beforeEach() { + Mockito.when(transport.getScheduler()).thenReturn(scheduler); + + Mockito.when(scheduler.schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any())) + .thenAnswer((InvocationOnMock iom) -> { + logger.debug("mock scheduled task"); + schedulerHelper.tasks.add(iom.getArgument(0, Runnable.class)); + return emptyFuture; + }); + } + + protected SchedulerAssert getScheduler() { + return schedulerHelper; + } + + protected StreamMock mockStream() { + StreamMock streamMock = new StreamMock(); + + GrpcReadWriteStream readWriteStream = Mockito.mock(GrpcReadWriteStream.class); + + Mockito.when(readWriteStream.start(Mockito.any())).thenAnswer( + (InvocationOnMock iom) -> { + streamMock.setObserver(iom.getArgument(0)); + return streamMock.streamFuture; + } + ).thenThrow(new RuntimeException("Unexpected second start call")); + + Mockito.doAnswer((Answer) (InvocationOnMock iom) -> { + streamMock.sent.add(iom.getArgument(0, SessionRequest.class)); + return null; + }).when(readWriteStream).sendNext(Mockito.any()); + + Mockito.when(transport.readWriteStreamCall(Mockito.eq(CoordinationServiceGrpc.getSessionMethod()), Mockito.any())) + .thenReturn(readWriteStream); + return streamMock; + } + + protected static class SchedulerAssert implements Executor { + private final Queue tasks = new ConcurrentLinkedQueue<>(); + + @Override + public void execute(@NotNull Runnable command) { + logger.debug("scheduling command: " + command); + tasks.add(command); + } + + public SchedulerAssert hasNoTasks() { + Assert.assertTrue(tasks.isEmpty()); + return this; + } + + public SchedulerAssert hasTasks(int count) { + Assert.assertEquals(count, tasks.size()); + return this; + } + + public SchedulerAssert executeNextTasks(int count) { + Assert.assertTrue(count <= tasks.size()); + + CompletableFuture.runAsync(() -> { + logger.debug("execute {} scheduled tasks", count); + for (int idx = 0; idx < count; idx++) { + tasks.poll().run(); + } + }).join(); + return this; + } + } + + protected class StreamMock { + private final CompletableFuture streamFuture; + private final List sent = new ArrayList<>(); + private volatile int sentIdx = 0; + + private volatile GrpcReadWriteStream.Observer observer = null; + + public StreamMock() { + streamFuture = new CompletableFuture<>(); + } + + public void setObserver(GrpcReadWriteStream.Observer observer) { + this.observer = observer; + } + + public void complete(StatusCode statusCode) { + streamFuture.complete(Status.of(statusCode)); + } + + public void complete(Status status) { + streamFuture.complete(status); + } + + public void complete(Throwable th) { + streamFuture.completeExceptionally(th); + } + + public void hasNoNewMessages() { + Assert.assertTrue(sentIdx >= sent.size()); + } + + public Checker nextMsg() { + Assert.assertTrue(sentIdx < sent.size()); + return new Checker(sent.get(sentIdx++)); + } + + public void responseSemaphoreAlreadyExists() { + SessionResponse msg = SessionResponse.newBuilder() + .setAcquireSemaphoreResult( + SessionResponse.AcquireSemaphoreResult.newBuilder().setStatus( + StatusCodesProtos.StatusIds.StatusCode.ALREADY_EXISTS + ) + ) + .build(); + response(msg); + } + + public void responseSessionStarted(long sessionId) { + SessionResponse msg = SessionResponse.newBuilder() + .setSessionStarted( + SessionResponse.SessionStarted.newBuilder() + .setSessionId(sessionId) + .build() + ) + .build(); + response(msg); + } + + public void responseAcquiredSuccessfully(long requestId) { + SessionResponse msg = SessionResponse.newBuilder() + .setAcquireSemaphoreResult( + SessionResponse.AcquireSemaphoreResult.newBuilder() + .setReqId(requestId) + .setAcquired(true) + .setStatus(StatusCodesProtos.StatusIds.StatusCode.SUCCESS) + ) + .build(); + response(msg); + } + + private void response(SessionResponse msg) { + Assert.assertNotNull(observer); + observer.onNext(msg); + } + + } + + protected class Checker { + private final SessionRequest msg; + + public Checker(SessionRequest msg) { + this.msg = msg; + } + + public SessionRequest get() { + return msg; + } + + public Checker isAcquireSemaphore() { + Assert.assertTrue("next msg must be acquire semaphore", msg.hasAcquireSemaphore()); + return this; + } + + public Checker isEphemeralSemaphore() { + Assert.assertTrue("next msg must be acquire ephemeral semaphore", msg.getAcquireSemaphore().getEphemeral()); + return this; + } + + public Checker hasSemaphoreName(String semaphoreName) { + Assert.assertEquals("invalid semaphore name", semaphoreName, msg.getAcquireSemaphore().getName()); + return this; + } + + public Checker isSessionStart() { + Assert.assertTrue("next msg must be session start", msg.hasSessionStart()); + return this; + } + + public Checker hasPath(String coordinationNodePath) { + Assert.assertEquals("invalid coordination node path", coordinationNodePath, msg.getSessionStart().getPath()); + return this; + } + } +} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java deleted file mode 100644 index cc3637c28..000000000 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/ReadWriteLockTest.java +++ /dev/null @@ -1,62 +0,0 @@ -package tech.ydb.coordination.recipes; - -import org.junit.Assert; -import org.junit.Test; - -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.CoordinationSessionBaseMockedTest; -import tech.ydb.coordination.SemaphoreLease; -import tech.ydb.coordination.settings.CoordinationSessionSettings; -import tech.ydb.core.Result; -import tech.ydb.core.StatusCode; - -import java.time.Duration; -import java.util.concurrent.CompletableFuture; - -public class ReadWriteLockTest extends CoordinationSessionBaseMockedTest { - - @Test - public void startTest() throws Exception { - StreamMock streamMock = mockStream(); - streamMock.complete(StatusCode.TRANSPORT_UNAVAILABLE); - - CoordinationSession session = client.createSession("/coordination/node/path"); - session.connect(); - streamMock.nextMsg().isSessionStart().hasPath("/coordination/node/path"); - - Assert.assertEquals(CoordinationSession.State.INITIAL, session.getState()); - } - - @Test - public void successAcquireTest() throws Exception { - StreamMock streamMock = mockStream(); - - CoordinationSession session = client.createSession( - "/coordination/node/path", - CoordinationSessionSettings.newBuilder() - .withExecutor(getScheduler()) - .build() - ); - session.connect(); - streamMock.nextMsg().isSessionStart().hasPath("/coordination/node/path"); - streamMock.responseSessionStarted(123); - - getScheduler().hasTasks(1).executeNextTasks(1); - Assert.assertEquals(CoordinationSession.State.CONNECTED, session.getState()); - - CompletableFuture> leaseResult = session.acquireEphemeralSemaphore("lock", false, Duration.ofSeconds(1)); - long requestId = streamMock.nextMsg().isAcquireSemaphore() - .isEphemeralSemaphore() - .hasSemaphoreName("lock") - .get() - .getAcquireSemaphore() - .getReqId(); - streamMock.responseAcquiredSuccessfully(requestId); - getScheduler().hasTasks(1).executeNextTasks(1); - Result lease = leaseResult.join(); - - Assert.assertTrue(lease.isSuccess()); - Assert.assertEquals("lock", lease.getValue().getSemaphoreName()); - } - -} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java new file mode 100644 index 000000000..097c746f9 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java @@ -0,0 +1,185 @@ +package tech.ydb.coordination.recipes.election; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BooleanSupplier; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.test.junit4.GrpcTransportRule; + +public class LeaderElectionIntegrationTest { + private static final Logger logger = LoggerFactory.getLogger(LeaderElectionIntegrationTest.class); + + @ClassRule + public static final GrpcTransportRule ydbRule = new GrpcTransportRule(); + + private static CoordinationClient client; + + @BeforeClass + public static void init() { + client = CoordinationClient.newClient(ydbRule); + } + + @AfterClass + public static void clean() { + ydbRule.close(); + } + + private LeaderElection getLeaderElector( + String testName, + LeaderElectionListener leaderElectionListener + ) { + return getLeaderElector(testName, testName, leaderElectionListener); + } + + private LeaderElection getLeaderElector( + String nodePath, + String lockName, + LeaderElectionListener leaderElectionListener + ) { + client.createNode(nodePath).join().expectSuccess("cannot create coordination path"); + LeaderElection leaderElectorImpl = new LeaderElection( + client, + leaderElectionListener, + nodePath, + lockName, + null, + Executors.newSingleThreadExecutor() + ); + return leaderElectorImpl; + } + + @Test + public void shouldCallTakeLeadershipWhenElected() throws Exception { + AtomicBoolean leadershipTaken = new AtomicBoolean(false); + + String testName = "shouldCallTakeLeadershipWhenElected"; + LeaderElection elector = getLeaderElector(testName, new LeaderElectionListener() { + @Override + public void takeLeadership() throws Exception { + leadershipTaken.set(true); + logger.debug("Leadership is taken"); + } + }); + elector.start(); + elector.requeue(); + + await(Duration.ofSeconds(10), Duration.ofMillis(100), leadershipTaken::get); + Assert.assertTrue(leadershipTaken.get()); + } + + @Test + public void shouldCallTakeLeadershipAgainAfterRequeue() throws Exception { + AtomicInteger leadershipCount = new AtomicInteger(0); + + String testName = "shouldCallTakeLeadershipAgainAfterRequeue"; + LeaderElection elector = getLeaderElector(testName, new LeaderElectionListener() { + @Override + public void takeLeadership() throws Exception { + leadershipCount.incrementAndGet(); + } + }); + elector.start(); + + elector.requeue(); + await(Duration.ofSeconds(10), Duration.ofMillis(100), () -> leadershipCount.get() > 0); + + elector.requeue(); + await(Duration.ofSeconds(10), Duration.ofMillis(100), () -> leadershipCount.get() > 1); + } + + @Test + @Ignore + public void shouldTrackParticipantsAndLeader() throws Exception { + String testName = "shouldTrackParticipantsAndLeader"; + + // Create first leader + AtomicBoolean leader1Taken = new AtomicBoolean(false); + LeaderElection elector1 = getLeaderElector(testName, new LeaderElectionListener() { + @Override + public void takeLeadership() throws Exception { + logger.info("Leadership 1 taken"); + leader1Taken.set(true); + Thread.sleep(5000); + logger.info("Leadership 1 ended"); + } + }); + elector1.start(); + elector1.requeue(); + + await(Duration.ofSeconds(10), Duration.ofMillis(100), leader1Taken::get); + + // Check participants and leader + List participants1 = elector1.getParticipants(); + Optional leader1 = elector1.getCurrentLeader(); + logger.info("current leader 1 {}", leader1); + logger.info("current participants 1 {}", participants1); + + Assert.assertEquals(1, participants1.size()); + Assert.assertTrue(leader1.isPresent()); + Assert.assertEquals(participants1.get(0).getSessionId(), leader1.get().getSessionId()); + + // Add second leader + AtomicBoolean leader2Taken = new AtomicBoolean(false); + LeaderElection elector2 = getLeaderElector(testName, new LeaderElectionListener() { + @Override + public void takeLeadership() throws Exception { + logger.info("Leadership 2 taken"); + leader2Taken.set(true); + Thread.sleep(20000); + logger.info("Leadership 2 ended"); + } + }); + elector2.start(); + elector2.requeue(); + + await(Duration.ofSeconds(10), Duration.ofMillis(100), leader2Taken::get); + // Check participants and leader + participants1 = elector1.getParticipants(); + leader1 = elector1.getCurrentLeader(); + logger.info("current leader 1 {}", leader1); + logger.info("current participants 1 {}", participants1); + + List participants2 = elector2.getParticipants(); + Optional leader2 = elector2.getCurrentLeader(); + logger.info("current leader 2 {}", leader2); + logger.info("current participants 2 {}", participants2); + + Assert.assertTrue(leader2Taken.get()); + Assert.assertTrue(elector2.isLeader()); + Assert.assertEquals(elector2.getCurrentLeader().get().getSessionId(), + elector1.getCurrentLeader().get().getSessionId()); + Assert.assertFalse(elector1.isLeader()); + } + + public static void await(Duration waitDuration, Duration checkDuration, BooleanSupplier condition) { + long timeout = System.currentTimeMillis() + waitDuration.toMillis(); + + while (System.currentTimeMillis() < timeout) { + if (condition.getAsBoolean()) { + return; + } + + try { + Thread.sleep(checkDuration.toMillis()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Await interrupted", e); + } + } + + throw new RuntimeException("Condition not met within " + waitDuration); + } +} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java new file mode 100644 index 000000000..e93181ad6 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java @@ -0,0 +1,66 @@ +package tech.ydb.coordination.recipes.group; + +import java.util.List; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.common.retry.RetryForever; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.test.junit4.GrpcTransportRule; + +public class GroupMembershipIntegrationTest { + private static final Logger logger = LoggerFactory.getLogger(GroupMembershipIntegrationTest.class); + + @ClassRule + public static final GrpcTransportRule ydbRule = new GrpcTransportRule(); + + private static CoordinationClient client; + + @BeforeClass + public static void init() { + client = CoordinationClient.newClient(ydbRule); + } + + @AfterClass + public static void clean() { + ydbRule.close(); + } + + private GroupMembershipImpl getGroupMembership(String testName) { + return getGroupMembership(testName, testName); + } + + private GroupMembershipImpl getGroupMembership( + String coordinationNodePath, + String groupName + ) { + client.createNode(coordinationNodePath).join().expectSuccess( + "cannot create coordination node on path: " + coordinationNodePath + ); + return new GroupMembershipImpl( + client, + coordinationNodePath, + groupName, + new RetryForever(100) + ); + } + + @Test + public void successTest() throws Exception { + String testName = "successTest"; + + GroupMembershipImpl groupMembership = getGroupMembership(testName); + groupMembership.start(); + + List currentMembers = groupMembership.getCurrentMembers(); + Assert.assertEquals(1, currentMembers.size()); + + groupMembership.close(); + } + +} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java new file mode 100644 index 000000000..e1bc4b1f6 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java @@ -0,0 +1,218 @@ +package tech.ydb.coordination.recipes.locks; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.test.junit4.GrpcTransportRule; + +public class InterProcessMutexIntegrationTest { + private static final Logger logger = LoggerFactory.getLogger(InternalLockMockedTest.class); + + @ClassRule + public static final GrpcTransportRule ydbRule = new GrpcTransportRule(); + + private static CoordinationClient client; + + @BeforeClass + public static void init() { + client = CoordinationClient.newClient(ydbRule); + } + + @AfterClass + public static void clean() { + ydbRule.close(); + } + + private InterProcessMutex getInterProcessMutex() { + return getInterProcessMutex(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + + private InterProcessMutex getInterProcessMutex(String testName) { + return getInterProcessMutex(testName, testName); + } + + private InterProcessMutex getInterProcessMutex(String nodePath, String lockName) { + client.createNode(nodePath).join().expectSuccess("cannot create coordination path"); + InterProcessMutex lock = new InterProcessMutex( + client, + nodePath, + lockName, + InterProcessMutexSettings.newBuilder() + .build() + ); + return lock; + } + + /** + * Asserts that code does not throw any exceptions + */ + @Test(timeout = 10000) + public void simpleLockTest() throws Exception { + InterProcessMutex lock = getInterProcessMutex(); + + lock.acquire(); + Assert.assertTrue(lock.isAcquiredInThisProcess()); + Thread.sleep(100); + lock.release(); + } + + /** + * Asserts that code does not throw any exceptions + */ + @Test(timeout = 10000) + public void tryLockTest() throws Exception { + String testName = "tryLockTest"; + InterProcessMutex lock1 = getInterProcessMutex(testName); + InterProcessMutex lock2 = getInterProcessMutex(testName); + + lock1.acquire(); + Assert.assertTrue(lock1.isAcquiredInThisProcess()); + + Assert.assertFalse(lock2.acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock2.isAcquiredInThisProcess()); + } + + /** + * Asserts that there is no data race around counter that is protected by distributed lock + * When locksN sessions tries to acquire lock at the same time + */ + @Test(timeout = 10000) + public void concurrentLockTest() { + String testName = "concurrentLockTest"; + // given + ExecutorService executor = Executors.newFixedThreadPool(2); + int cycles = 5; + int locksN = 5; + + List locks = new ArrayList<>(locksN); + for (int i = 0; i < locksN; i++) { + locks.add(getInterProcessMutex(testName)); + } + + AtomicInteger counter = new AtomicInteger(0); + + // when + List> tasks = locks.stream().map(lock -> + (Callable) () -> { + for (int i = 0; i < cycles; i++) { + lock.acquire(); + int start = counter.get(); + logger.debug("Lock acquired, cycle = {}, count = {}", i, start); + Thread.sleep(100); + counter.set(start + 1); + logger.debug("Lock released, cycle = {}", i); + lock.release(); + } + return null; + } + ).collect(Collectors.toList()); + + try { + List> futures = executor.invokeAll(tasks); + futures.forEach(future -> { + try { + future.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } catch (Exception ignored) { + } + + // then + Assert.assertEquals(cycles * locksN, counter.get()); + + executor.shutdown(); + } + + @Test(timeout = 10000) + public void reentrantLockTest() throws Exception { + InterProcessMutex lock = getInterProcessMutex(); + + // first acquire + lock.acquire(); + Assert.assertTrue(lock.isAcquiredInThisProcess()); + + // try to acquire the same lock + Assert.assertThrows(LockAlreadyAcquiredException.class, lock::acquire); + Assert.assertTrue(lock.isAcquiredInThisProcess()); + Assert.assertTrue(lock.release()); + } + + @Test(timeout = 10000) + public void longWaitTimeoutTest() throws Exception { + String testName = "reentrantLockTest"; + InterProcessMutex lock1 = getInterProcessMutex(testName); + InterProcessMutex lock2 = getInterProcessMutex(testName); + + lock1.acquire(); + Assert.assertFalse(lock2.acquire(Duration.ofMillis(10))); + } + + @Test(timeout = 10000) + public void releaseNotAcquiredLockTest() throws Exception { + InterProcessMutex lock = getInterProcessMutex(); + + Assert.assertFalse(lock.release()); + } + + @Test(timeout = 10000) + public void concurrentReleaseTest() throws Exception { + ExecutorService executor = Executors.newFixedThreadPool(2); + InterProcessMutex lock = getInterProcessMutex(); + + lock.acquire(); + + Future future1 = executor.submit(lock::release); + Future future2 = executor.submit(lock::release); + + // Only one concurrent release must be successful + Assert.assertTrue(future1.get() ^ future2.get()); + + executor.shutdown(); + } + +} + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java new file mode 100644 index 000000000..d8e8b7240 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java @@ -0,0 +1,572 @@ +package tech.ydb.coordination.recipes.locks; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.junit.Assert; +import org.junit.Test; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.CoordinationSessionBaseMockedTest; +import tech.ydb.core.StatusCode; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class InternalLockMockedTest extends CoordinationSessionBaseMockedTest { + + /* + Части функционала: + tryAcquire + isAcquired (deadline, data, exclusive) + release + isAcquired (уже освобожден, еще не захвачен?) + + listeners + listeners + reconnection when session is interrupted + + session start + close - (освобождает ресурсы) + + Внешние эффекты: + 1) состояние сессии (изначальное, в процессе) + 1.1) Дожидается открытия соединения + 1.2) Разрыв соединения (LOST, CLOSED) - возвращает сразу ошибку + 2) ответы на запросы + 2.1) При ошибке ретраит то что можно + 2.2) Кидает ошибку, если не ретраится + 3) прерывание потока + 4) контракт: вызвать 2 раза, 0 раз + + acquire: + Критерии корректности: + 1) lock.isAcquired - нету нарушения внутреннего состояния (при успехе и не успехе) + 2) был успешный вызов session.isAcquired + + Тестовые кейсы: + 1) Все ОК - successAcquireTest + 2) Сессия сразу разорвана - failedAcquireOnLostSessionTest + 3) Сессия рвется в процессе захвата - failedAcquireDuringSessionLostTest + 4) Сессия рвется после захвата - successAcquireThenReleasedOnLostSessionTest + 4) При ответе ретраит - acquireRetriedOnRetryableTest + 5) При ответе ошибки кидает ошибку - acquireFailsOnNonRetryableErrorTest + 6) На блокировке может быть прервано - acquireInterruptedTest + 7) Уже захвачен - acquireFailsAlreadyAcquiredTest + 8) Все ОК (с таймаутом) - successTryAcquireTest + 9) Таймаут вышел - tryAcquireFailsTimeoutExceededTest + + release: + Критерии корректности: + 1) lock.isAcquired - нету нарушения внутреннего состояния (при успехе и не успехе) + 2) был успешный вызов releaseSemaphore + + Тестовые кейсы: + 1) Все ОК (сессия, захват, освобождение) - release_RespondedStatusSuccess_ReleasedLock + 2) Семафор не был даже захвачен - release_NoCurrentLease_ReturnedFalse + 5) Вызван поверх LOST сессии - release_AlreadySessionLost_ThrowsLockReleaseFailedException + 3) Порвалась сессия в процессе освобождения - release_DuringSessionLost_ThrowsLockReleaseFailedException + 4) Корректно обрабатывается InterruptedException - release_Interrupted_StateConsistent + + */ + + @Test + public void acquire_RespondedStatusSuccess_AcquiresLock() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + LeaseMock lease = lease("lock_name"); + sessionMock.acquireEphemeralSemaphore() + .then(successAcquire(lease)); + + LockInternals.LeaseData leaseData = lock.tryAcquire(null, false, null); + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + verify(getCoordinationSession()) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + + lock.close(); + } + + @Test + public void acquire_WithCustomData_PropagatesToSession() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect().then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + LeaseMock lease = lease("lock_name"); + sessionMock.acquireEphemeralSemaphore() + .then(successAcquire(lease)); + + byte[] testData = "test_payload".getBytes(StandardCharsets.UTF_8); + LockInternals.LeaseData leaseData = lock.tryAcquire(null, true, testData); + + Assert.assertNotNull(leaseData); + Assert.assertTrue(leaseData.isExclusive()); + verify(getCoordinationSession()).acquireEphemeralSemaphore( + eq("lock_name"), + eq(true), // exclusive + eq(testData), + any() // deadline + ); + } + + + @Test + public void acquire_LostSession_ThrowsLockAcquireFailedException() { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + verify(getCoordinationSession()) + .connect(); + + sessionMock.lost(); + + Assert.assertThrows( + LockAcquireFailedException.class, + () -> lock.tryAcquire(null, false, null) + ); + Assert.assertFalse(lock.isAcquired()); + } + + @Test + public void acquire_SessionLostDuringBlock_ThrowsLockAcquireFailedException() { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + verify(getCoordinationSession()) + .connect(); + + sessionMock.connected(); + + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire()) + .then(lostAcquire()) + .then(successAcquire(lease("lock_name"))); // never reached + + Assert.assertThrows( + LockAcquireFailedException.class, + () -> lock.tryAcquire(null, false, null) + ); + Assert.assertFalse(lock.isAcquired()); + verify(getCoordinationSession(), times(2)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + } + + @Test + public void acquire_RespondedSuccessStatusThenLostSession_ReleasedLock() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + lock.getSessionListenable().addListener( + getSessionStateAssert() + .next(CoordinationSession.State.CONNECTING) + .next(CoordinationSession.State.CONNECTED) + .next(CoordinationSession.State.LOST) + ); + + sessionMock.connecting(); + verify(getCoordinationSession()) + .connect(); + sessionMock.connected(); + + sessionMock.acquireEphemeralSemaphore() + .then(successAcquire(lease("lock_name"))); + + LockInternals.LeaseData leaseData = lock.tryAcquire(null, false, null); + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + verify(getCoordinationSession()) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + + sessionMock.lost(); + Assert.assertFalse(lock.isAcquired()); + getSessionStateAssert().finished(); + } + + @Test + public void acquire_RespondedRetryableStatusThenSuccessStatus_AcquiredLock() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + verify(getCoordinationSession()) + .connect(); + + sessionMock.connected(); + + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire()) + .then(statusAcquire(StatusCode.SESSION_BUSY)) + .then(statusAcquire(StatusCode.UNAVAILABLE)) + .then(statusAcquire(StatusCode.TRANSPORT_UNAVAILABLE)) + .then(successAcquire(lease("lock_name"))); + + LockInternals.LeaseData leaseData = lock.tryAcquire(null, false, null); + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + verify(getCoordinationSession(), times(5)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + ; + } + + @Test + public void acquire_RespondedNonRetryableStatus_ThrowsLockAcquireFailedException() { + StatusCode badStatus = StatusCode.BAD_REQUEST; + + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + verify(getCoordinationSession()) + .connect(); + + sessionMock.connected(); + + sessionMock.acquireEphemeralSemaphore() + .then(statusAcquire(badStatus)) + .then(successAcquire(lease("lock_name"))); // never reached + + Assert.assertThrows( + LockAcquireFailedException.class, + () -> lock.tryAcquire(null, false, null) + ); + + Assert.assertFalse(lock.isAcquired()); + verify(getCoordinationSession(), times(1)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + } + + @Test + public void acquire_BlockingInterrupted_ThrowsInterruptedException() throws InterruptedException { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + verify(getCoordinationSession()) + .connect(); + + sessionMock.connected(); + + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire(Duration.ofSeconds(120))) + .then(successAcquire(lease("lock_name"))); // never reached + + CountDownLatch latch = new CountDownLatch(1); + ExecutorService executor = Executors.newSingleThreadExecutor(); + Future future = executor.submit(() -> { + latch.countDown(); + Assert.assertThrows( + InterruptedException.class, + () -> lock.tryAcquire(null, false, null) + ); + }); + + Thread.sleep(20); + latch.await(); + future.cancel(true); + + Assert.assertFalse(lock.isAcquired()); + verify(getCoordinationSession(), times(1)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + } + + @Test + public void acquire_CallAcquireTwice_ThrowsLockAlreadyAcquiredException() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + verify(getCoordinationSession()) + .connect(); + + sessionMock.connected(); + + sessionMock.acquireEphemeralSemaphore() + .then(successAcquire(lease("lock_name"))) + .then(statusAcquire(StatusCode.BAD_REQUEST)); // never reached + + LockInternals.LeaseData leaseData = lock.tryAcquire(null, false, null); + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + + verify(getCoordinationSession(), times(1)) + .acquireEphemeralSemaphore(any(), anyBoolean(), any(), any()); + + Assert.assertThrows( + LockAlreadyAcquiredException.class, + () -> lock.tryAcquire(null, false, null) + ); + Assert.assertTrue(lock.isAcquired()); + verify(getCoordinationSession(), times(1)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + } + + @Test + public void acquireWithTimeout_RespondedSuccessStatus_AcquiredLock() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + LeaseMock lease = lease("lock_name"); + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire(Duration.ofMillis(100))) + .then(successAcquire(lease)); + + Instant deadline = Instant.now().plus(Duration.ofMillis(1000)); + LockInternals.LeaseData leaseData = lock.tryAcquire(deadline, false, null); + + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + verify(getCoordinationSession(), times(2)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + } + + @Test + public void acquireWithTimeout_ResponseTimeout_ReturnsFalse() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + LeaseMock lease = lease("lock_name"); + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire(Duration.ofMillis(100))) + .then(successAcquire(lease)); // never reaches + + Instant deadline = Instant.now().plus(Duration.ofMillis(10)); + LockInternals.LeaseData leaseData = lock.tryAcquire(deadline, false, null); + + Assert.assertNull(leaseData); + Assert.assertFalse(lock.isAcquired()); + verify(getCoordinationSession(), times(1)) + .acquireEphemeralSemaphore( + eq("lock_name"), + eq(false), + isNull(), + any() + ); + } + + @Test + public void release_RespondedStatusSuccess_ReleasedLock() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + LeaseMock lease = lease("lock_name"); + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire(Duration.ofMillis(100))) + .then(successAcquire(lease)); + + LockInternals.LeaseData leaseData = lock.tryAcquire(null, false, null); + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + + Assert.assertTrue(lock.release()); + Assert.assertFalse(lock.isAcquired()); + lease.assertReleased(); + } + + @Test + public void release_NoCurrentLease_ReturnedFalse() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + Assert.assertFalse(lock.release()); + Assert.assertFalse(lock.isAcquired()); + } + + @Test + public void release_AlreadySessionLost_ThrowsLockReleaseFailedException() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + + sessionMock.lost(); + + Assert.assertThrows(LockReleaseFailedException.class, lock::release); + Assert.assertFalse(lock.isAcquired()); + } + + @Test + public void release_DuringSessionLost_ThrowsLockReleaseFailedException() throws Exception { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + LockInternals lock = new LockInternals( + getClient(), + "/node/path", + "lock_name" + ); + lock.start(); + sessionMock.connected(); + + LeaseMock lease = lease("lock_name") + .failed(new IllegalStateException()); + sessionMock.acquireEphemeralSemaphore() + .then(timeoutAcquire(Duration.ofMillis(100))) + .then(successAcquire(lease)); + + LockInternals.LeaseData leaseData = lock.tryAcquire(null, false, null); + Assert.assertNotNull(leaseData); + Assert.assertFalse(leaseData.isExclusive()); + Assert.assertTrue(lock.isAcquired()); + + Assert.assertThrows(LockReleaseFailedException.class, lock::release); + lease.assertReleased(); + } + +} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java new file mode 100644 index 000000000..d7e17a9c3 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java @@ -0,0 +1,302 @@ +package tech.ydb.coordination.recipes.locks; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.test.junit4.GrpcTransportRule; + +public class ReadWriteInterProcessLockIntegrationTest { + private static final Logger logger = LoggerFactory.getLogger(ReadWriteInterProcessLockIntegrationTest.class); + + @ClassRule + public static final GrpcTransportRule ydbRule = new GrpcTransportRule(); + + private static CoordinationClient client; + + @BeforeClass + public static void init() { + client = CoordinationClient.newClient(ydbRule); + } + + @AfterClass + public static void clean() { + ydbRule.close(); + } + + /** + * Asserts that code does not throw any exceptions for basic write lock operations + */ + @Test + public void simpleWriteLockTest() throws Exception { + ReadWriteInterProcessLock rwLock = getReadWriteLock(); + + rwLock.writeLock().acquire(); + Assert.assertTrue(rwLock.writeLock().isAcquiredInThisProcess()); + Thread.sleep(100); + rwLock.writeLock().release(); + } + + /** + * Asserts that code does not throw any exceptions for basic read lock operations + */ + @Test + public void simpleReadLockTest() throws Exception { + ReadWriteInterProcessLock rwLock = getReadWriteLock(); + + rwLock.readLock().acquire(); + Assert.assertTrue(rwLock.readLock().isAcquiredInThisProcess()); + Thread.sleep(100); + rwLock.readLock().release(); + } + + /** + * Tests that write lock is exclusive (only one can acquire it) + */ + @Test + public void writeLockExclusivityTest() throws Exception { + String coordinationNodePath = "writeLockExclusivityTest"; + String lockName = "writeLockExclusivityTest"; + ReadWriteInterProcessLock lock1 = getReadWriteLock(coordinationNodePath, lockName); + ReadWriteInterProcessLock lock2 = getReadWriteLock(coordinationNodePath, lockName); + + lock1.writeLock().acquire(); + Assert.assertTrue(lock1.writeLock().isAcquiredInThisProcess()); + + Assert.assertFalse(lock2.writeLock().acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock2.writeLock().isAcquiredInThisProcess()); + } + + /** + * Tests that multiple read locks can be acquired simultaneously + */ + @Test + public void readLockSharedAccessTest() throws Exception { + String coordinationNodePath = "readLockSharedAccessTest"; + String lockName = "readLockSharedAccessTest"; + ReadWriteInterProcessLock lock1 = getReadWriteLock(coordinationNodePath, lockName); + ReadWriteInterProcessLock lock2 = getReadWriteLock(coordinationNodePath, lockName); + + lock1.readLock().acquire(); + Assert.assertTrue(lock1.readLock().isAcquiredInThisProcess()); + + Assert.assertTrue(lock2.readLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(lock2.readLock().isAcquiredInThisProcess()); + } + + + /** + * Tests that write lock cannot be acquired while read lock is held + */ + @Test + public void writeLockBlockedByReadLockTest() throws Exception { + String coordinationNodePath = "writeLockBlockedByReadLockTest"; + String lockName = "writeLockBlockedByReadLockTest"; + ReadWriteInterProcessLock lock1 = getReadWriteLock(coordinationNodePath, lockName); + ReadWriteInterProcessLock lock2 = getReadWriteLock(coordinationNodePath, lockName); + + lock1.readLock().acquire(); + Assert.assertTrue(lock1.readLock().isAcquiredInThisProcess()); + + Assert.assertFalse(lock2.writeLock().acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock2.writeLock().isAcquiredInThisProcess()); + } + + /** + * Tests that read lock cannot be acquired while write lock is held + */ + @Test + public void readLockBlockedByWriteLockTest() throws Exception { + String coordinationNodePath = "readLockBlockedByWriteLockTest"; + String lockName = "readLockBlockedByWriteLockTest"; + ReadWriteInterProcessLock lock1 = getReadWriteLock(coordinationNodePath, lockName); + ReadWriteInterProcessLock lock2 = getReadWriteLock(coordinationNodePath, lockName); + + lock1.writeLock().acquire(); + Assert.assertTrue(lock1.writeLock().isAcquiredInThisProcess()); + + Assert.assertFalse(lock2.readLock().acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock2.readLock().isAcquiredInThisProcess()); + } + + /** + * Concurrent test for write locks (should be exclusive) + */ + @Test + public void concurrentWriteLockTest() { + // given + ExecutorService executor = Executors.newFixedThreadPool(2); + int cycles = 10; + int locksN = 10; + + String nodePath = UUID.randomUUID().toString(); + String lockName = UUID.randomUUID().toString(); + List locks = new ArrayList<>(locksN); + for (int i = 0; i < locksN; i++) { + locks.add(getReadWriteLock(nodePath, lockName)); + } + + AtomicInteger counter = new AtomicInteger(0); + + // when + List> tasks = locks.stream().map(lock -> + (Callable) () -> { + for (int i = 0; i < cycles; i++) { + lock.writeLock().acquire(); + int start = counter.get(); + logger.debug("Write lock acquired, cycle = {}, count = {}", i, start); + Thread.sleep(100); + counter.set(start + 1); // not an atomic increment + logger.debug("Write lock released, cycle = {}", i); + lock.writeLock().release(); + } + return null; + } + ).collect(Collectors.toList()); + + try { + List> futures = executor.invokeAll(tasks); + futures.forEach(future -> { + try { + future.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } catch (Exception ignored) { + } + + // then + Assert.assertEquals(cycles * locksN, counter.get()); + executor.shutdown(); + } + + /** + * Mixed read-write lock test with more complex scenarios + */ + @Test + public void mixedReadWriteLockTest() throws Exception { + String coordinationNodePath = "mixedReadWriteLockTest"; + String lockName = "mixedReadWriteLockTest"; + ReadWriteInterProcessLock lock1 = getReadWriteLock(coordinationNodePath, lockName); + ReadWriteInterProcessLock lock2 = getReadWriteLock(coordinationNodePath, lockName); + ReadWriteInterProcessLock lock3 = getReadWriteLock(coordinationNodePath, lockName); + + // 1. Test read lock sharing + lock1.readLock().acquire(); + Assert.assertTrue(lock1.readLock().isAcquiredInThisProcess()); + + // Second read lock should be allowed + Assert.assertTrue(lock2.readLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(lock2.readLock().isAcquiredInThisProcess()); + + // Write lock should be blocked while read locks are held + Assert.assertFalse(lock3.writeLock().acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock3.writeLock().isAcquiredInThisProcess()); + + // 2. Release read locks and test write lock exclusivity + lock1.readLock().release(); + lock2.readLock().release(); + + // Now write lock should be acquirable + Assert.assertTrue(lock3.writeLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(lock3.writeLock().isAcquiredInThisProcess()); + + // Read locks should be blocked while write lock is held + Assert.assertFalse(lock1.readLock().acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock1.readLock().isAcquiredInThisProcess()); + Assert.assertFalse(lock2.readLock().acquire(Duration.ofMillis(100))); + Assert.assertFalse(lock2.readLock().isAcquiredInThisProcess()); + + // 3. Release write lock and test read lock acquisition again + lock3.writeLock().release(); + + // Read locks should be acquirable again + Assert.assertTrue(lock1.readLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(lock1.readLock().isAcquiredInThisProcess()); + Assert.assertTrue(lock2.readLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(lock2.readLock().isAcquiredInThisProcess()); + + // 4. Test write lock waiting for read locks to be released + ExecutorService executor = Executors.newSingleThreadExecutor(); + Future writeLockFuture = executor.submit(() -> { + // This should block until read locks are released + return lock3.writeLock().acquire(Duration.ofSeconds(2)); + }); + + // Wait a bit to ensure write lock is waiting + Thread.sleep(100); + Assert.assertFalse(writeLockFuture.isDone()); + + // Release read locks + lock1.readLock().release(); + lock2.readLock().release(); + + // Now write lock should be acquired + Assert.assertTrue(writeLockFuture.get(1, TimeUnit.SECONDS)); + Assert.assertTrue(lock3.writeLock().isAcquiredInThisProcess()); + + // Cleanup + executor.shutdown(); + } + + /** + * Test re-acquisition after release + */ + @Test + public void reacquisitionTest() throws Exception { + ReadWriteInterProcessLock rwLock = getReadWriteLock(); + + // 1. Test write lock re-acquisition + rwLock.writeLock().acquire(); + Assert.assertTrue(rwLock.writeLock().isAcquiredInThisProcess()); + rwLock.writeLock().release(); + Assert.assertFalse(rwLock.writeLock().isAcquiredInThisProcess()); + + // Should be able to acquire again + Assert.assertTrue(rwLock.writeLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(rwLock.writeLock().isAcquiredInThisProcess()); + rwLock.writeLock().release(); + + // 2. Test read lock re-acquisition + rwLock.readLock().acquire(); + Assert.assertTrue(rwLock.readLock().isAcquiredInThisProcess()); + rwLock.readLock().release(); + Assert.assertFalse(rwLock.readLock().isAcquiredInThisProcess()); + + // Should be able to acquire again + Assert.assertTrue(rwLock.readLock().acquire(Duration.ofMillis(100))); + Assert.assertTrue(rwLock.readLock().isAcquiredInThisProcess()); + rwLock.readLock().release(); + } + + private ReadWriteInterProcessLock getReadWriteLock() { + return getReadWriteLock(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + + private ReadWriteInterProcessLock getReadWriteLock(String nodePath, String lockName) { + client.createNode(nodePath).join().expectSuccess("cannot create coordination path"); + return new ReadWriteInterProcessLock( + client, + nodePath, + lockName + ); + } + +} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java new file mode 100644 index 000000000..8ccc7638b --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java @@ -0,0 +1,29 @@ +package tech.ydb.coordination.recipes.util; + +import org.junit.Test; +import tech.ydb.common.retry.RetryForever; +import tech.ydb.coordination.CoordinationSessionBaseMockedTest; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.settings.WatchSemaphoreMode; + +public class SemaphoreObserverMockedTest extends CoordinationSessionBaseMockedTest { + + @Test + public void successTest() { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + + SemaphoreObserver observer = new SemaphoreObserver( + getCoordinationSession(), + "observable_semaphore", + WatchSemaphoreMode.WATCH_DATA_AND_OWNERS, + DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, + new RetryForever(100) + ); + observer.start(); + sessionMock.connected(); + observer.getCachedData(); + } + +} From 082be540be0bc2c37217d1178990ec33cbc72e7c Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Mon, 5 May 2025 20:45:54 +0300 Subject: [PATCH 04/10] finished interprocess mutex --- .../recipes/election/LeaderElection.java | 83 ++++--- .../election/LeaderElectionSettings.java | 54 +++++ .../recipes/group/GroupMembershipImpl.java | 2 +- .../group/GroupMembershipSettings.java | 4 + .../recipes/locks/InterProcessLock.java | 40 +++- .../recipes/locks/InterProcessMutex.java | 168 ++++++++++++- .../locks/InterProcessMutexSettings.java | 1 - .../locks/LockAcquireFailedException.java | 28 --- .../locks/LockAlreadyAcquiredException.java | 11 - .../recipes/locks/LockInternals.java | 192 +++++---------- .../locks/LockReleaseFailedException.java | 38 --- .../locks/ReadWriteInterProcessLock.java | 77 ++++-- .../ReadWriteInterProcessLockSettings.java | 31 +++ .../exception/LockAcquireFailedException.java | 18 ++ .../LockAlreadyAcquiredException.java | 18 ++ .../locks/exception/LockException.java | 23 ++ .../LockInitializationException.java | 18 ++ .../exception/LockReleaseFailedException.java | 18 ++ .../locks/exception/LockStateException.java | 19 ++ .../CoordinationSessionBaseMockedTest.java | 28 --- .../CoordinationStreamBaseMockedTest.java | 223 ------------------ .../LeaderElectionIntegrationTest.java | 15 +- .../InterProcessMutexIntegrationTest.java | 2 + .../recipes/locks/InternalLockMockedTest.java | 90 ++----- 24 files changed, 594 insertions(+), 607 deletions(-) create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockSettings.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAcquireFailedException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAlreadyAcquiredException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockInitializationException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockReleaseFailedException.java create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockStateException.java delete mode 100644 coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java index d98e5418f..57532f632 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -5,8 +5,8 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -16,36 +16,40 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import tech.ydb.common.retry.RetryForever; import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.description.SemaphoreDescription; import tech.ydb.coordination.recipes.locks.LockInternals; import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableContainer; import tech.ydb.coordination.recipes.util.SessionListenableProvider; import tech.ydb.coordination.recipes.util.SemaphoreObserver; import tech.ydb.coordination.settings.DescribeSemaphoreMode; import tech.ydb.coordination.settings.WatchSemaphoreMode; +import tech.ydb.core.Status; import tech.ydb.core.StatusCode; -// TODO: настройки + переименовать переменные + рекомендации по коду + логгирование + backoff политика +// TODO: backoff политика + документцаия / логгирование / рекомендации по коду public class LeaderElection implements Closeable, SessionListenableProvider { private static final Logger logger = LoggerFactory.getLogger(LeaderElection.class); private static final long MAX_LEASE = 1L; - private final CoordinationClient client; private final LeaderElectionListener leaderElectionListener; private final String coordinationNodePath; private final String electionName; private final byte[] data; + private final ExecutorService electionExecutor; + private final CoordinationSession coordinationSession; + private final ListenableContainer sessionListenable; private final LockInternals lock; private final SemaphoreObserver semaphoreObserver; private AtomicReference state = new AtomicReference<>(State.CREATED); + private Future sessionConnectionTask = null; + private Future electionTask = null; private volatile boolean autoRequeue = false; private volatile boolean isLeader = false; - private Future electionTask = null; private enum State { CREATED, @@ -55,62 +59,77 @@ private enum State { public LeaderElection( CoordinationClient client, - LeaderElectionListener leaderElectionListener, String coordinationNodePath, String electionName, - byte[] data + byte[] data, + LeaderElectionListener leaderElectionListener ) { this( client, - leaderElectionListener, coordinationNodePath, electionName, data, - Executors.newSingleThreadExecutor() + leaderElectionListener, + LeaderElectionSettings.newBuilder() + .build() ); } public LeaderElection( CoordinationClient client, - LeaderElectionListener leaderElectionListener, String coordinationNodePath, String electionName, byte[] data, - ExecutorService executorService + LeaderElectionListener leaderElectionListener, + LeaderElectionSettings settings ) { - this.client = client; - this.leaderElectionListener = leaderElectionListener; this.coordinationNodePath = coordinationNodePath; this.electionName = electionName; this.data = data; - this.electionExecutor = executorService; + this.leaderElectionListener = leaderElectionListener; + this.electionExecutor = settings.getExecutorService(); + + this.coordinationSession = client.createSession(coordinationNodePath); + this.sessionListenable = new ListenableContainer<>(); + coordinationSession.addStateListener(sessionListenable::notifyListeners); this.lock = new LockInternals( - MAX_LEASE, - client, - coordinationNodePath, - electionName + coordinationSession, + electionName, + MAX_LEASE ); this.semaphoreObserver = new SemaphoreObserver( - lock.getCoordinationSession(), + coordinationSession, electionName, WatchSemaphoreMode.WATCH_OWNERS, DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, - new RetryForever(100) // TODO: передавать снаружи + settings.getRetryPolicy() ); } + private CoordinationSession connectedSession() { + if (sessionConnectionTask == null) { + throw new IllegalStateException("Not started yet"); + } + try { + sessionConnectionTask.get().expectSuccess("Unable to connect to session"); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + return coordinationSession; + } + public void start() { Preconditions.checkState(state.compareAndSet(State.CREATED, State.STARTED), "Already started or closed"); - // TODO: create session? - CoordinationSession coordinationSession = lock.getCoordinationSession(); - // TODO: retry on create? Non idempotent - will not be retried automatically - lock.start(); - coordinationSession.createSemaphore(electionName, MAX_LEASE).thenAccept(status -> { - if (status.isSuccess() || status.getCode() == StatusCode.ALREADY_EXISTS) { - semaphoreObserver.start(); - } - status.expectSuccess("Unable to create semaphore"); - // TODO: set status == error + // TODO: handle errors retries and logging? + this.sessionConnectionTask = coordinationSession.connect().thenCompose(connectionStatus -> { + connectionStatus.expectSuccess("Unable to establish session"); + return coordinationSession.createSemaphore(electionName, MAX_LEASE).thenApply(semaphoreStatus -> { + if (semaphoreStatus.isSuccess() || semaphoreStatus.getCode() == StatusCode.ALREADY_EXISTS) { + semaphoreObserver.start(); + } + semaphoreStatus.expectSuccess("Unable to create semaphore"); + return semaphoreStatus; + }); }); if (autoRequeue) { @@ -173,7 +192,6 @@ private void doWork() throws Exception { isLeader = false; try { - lock.getConnectedCoordinationSession(); // asserts that session is connected or throws exception lock.tryAcquire( null, true, @@ -221,6 +239,7 @@ private boolean isQueued() { /** * Не гарантированы все, кроме лидера + * * @return */ public List getParticipants() { @@ -257,7 +276,7 @@ private static ElectionParticipant mapParticipant(SemaphoreDescription.Session s @Override public Listenable getSessionListenable() { - return lock.getSessionListenable(); + return sessionListenable; } @Override diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java new file mode 100644 index 000000000..9507c90cd --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java @@ -0,0 +1,54 @@ +package tech.ydb.coordination.recipes.election; + +import java.time.Duration; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.common.retry.RetryUntilElapsed; + +public class LeaderElectionSettings { + public static final Duration DEFAULT_CONNECT_TIMEOUT = Duration.ofSeconds(5); + public static final RetryUntilElapsed DEFAULT_RETRY_POLICY = new RetryUntilElapsed( + DEFAULT_CONNECT_TIMEOUT.toMillis(), 250, 5 + ); + + private final ExecutorService executorService; + private final RetryPolicy retryPolicy; + + public LeaderElectionSettings(Builder builder) { + this.executorService = builder.executorService; + this.retryPolicy = builder.retryPolicy; + } + + public ExecutorService getExecutorService() { + return executorService; + } + + public RetryPolicy getRetryPolicy() { + return retryPolicy; + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private ExecutorService executorService = Executors.newSingleThreadExecutor(); + private RetryPolicy retryPolicy = DEFAULT_RETRY_POLICY; + + public Builder withExecutorService(ExecutorService executorService) { + this.executorService = executorService; + return this; + } + + public Builder withRetryPolicy(RetryPolicy retryPolicy) { + this.retryPolicy = retryPolicy; + return this; + } + + public LeaderElectionSettings build() { + return new LeaderElectionSettings(this); + } + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java index cb17b9942..db584407c 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java @@ -77,7 +77,7 @@ public void start() { session.connect().thenAccept(sessionStatus -> { sessionStatus.expectSuccess("Unable to establish session"); session.createSemaphore(groupName, MAX_GROUP_SIZE).thenAccept(semaphoreStatus -> { - lockInternals.start(); + // TODO: start acquiring task semaphoreObserver.start(); }); }); diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java new file mode 100644 index 000000000..23d8dbe2d --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java @@ -0,0 +1,4 @@ +package tech.ydb.coordination.recipes.group; + +public class GroupMembershipSettings { +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java index e1b7737b0..813649541 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java @@ -2,23 +2,51 @@ import java.time.Duration; +import tech.ydb.coordination.recipes.locks.exception.LockAcquireFailedException; +import tech.ydb.coordination.recipes.locks.exception.LockAlreadyAcquiredException; +import tech.ydb.coordination.recipes.locks.exception.LockReleaseFailedException; +import tech.ydb.coordination.recipes.locks.exception.LockStateException; import tech.ydb.coordination.recipes.util.SessionListenableProvider; public interface InterProcessLock extends SessionListenableProvider { - void acquire() throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException; + /** + * Acquires the distributed lock, blocking until it is obtained. + * + * @throws Exception if an unexpected error occurs + * @throws LockAlreadyAcquiredException if the lock is already acquired by this process + * @throws LockAcquireFailedException if the lock acquisition fails + * @throws LockStateException if the lock is in invalid state for acquisition + */ + void acquire() throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException, LockStateException; /** - * @return true - if successfully acquired lock, false - if lock waiting time expired + * Attempts to acquire the lock within the given waiting time. + * + * @param waitDuration maximum time to wait for the lock + * @return true if the lock was acquired, false if the waiting time elapsed + * @throws Exception if an unexpected error occurs + * @throws LockAlreadyAcquiredException if the lock is already acquired by this process + * @throws LockAcquireFailedException if the lock acquisition fails + * @throws LockStateException if the lock is in invalid state for acquisition */ - boolean acquire(Duration waitDuration) throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException; + boolean acquire(Duration waitDuration) throws Exception, LockAlreadyAcquiredException, LockAcquireFailedException, + LockStateException; /** - * @return false if nothing to release + * Releases the lock if it is held by this process. + * + * @return false if there was nothing to release + * @throws InterruptedException if the thread is interrupted + * @throws LockReleaseFailedException if the lock release fails + * @throws LockStateException if the lock is in invalid state for release */ - boolean release() throws InterruptedException, LockReleaseFailedException; + boolean release() throws InterruptedException, LockReleaseFailedException, LockStateException; + /** - * @return true if the lock is acquired by a thread in this JVM + * Checks if the lock is currently acquired by this process. + * + * @return true if the lock is acquired by this process */ boolean isAcquiredInThisProcess(); } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java index 2167d2529..ee79c78b9 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java @@ -3,64 +3,210 @@ import java.io.Closeable; import java.time.Duration; import java.time.Instant; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.locks.exception.LockException; +import tech.ydb.coordination.recipes.locks.exception.LockInitializationException; +import tech.ydb.coordination.recipes.locks.exception.LockStateException; import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableContainer; -// TODO: add documentation and logs +/** + * Distributed mutex implementation using a coordination service. + * This implementation is thread-safe and supports session state listening. + */ public class InterProcessMutex implements InterProcessLock, Closeable { + private static final Logger logger = LoggerFactory.getLogger(InterProcessMutex.class); + + private final AtomicReference state = new AtomicReference<>(State.INITIAL); + private final CoordinationSession coordinationSession; + private final Future sessionConnectionTask; private final LockInternals lockInternals; + private final ListenableContainer sessionListenable; + + /** + * Internal state machine states + */ + private enum State { + INITIAL, + STARTING, + STARTED, + FAILED, + CLOSED + } + + /** + * Creates a new distributed mutex instance with default settings. + * + * @param client coordination client + * @param coordinationNodePath path to the coordination node + * @param lockName name of the lock + * @throws IllegalArgumentException if any parameter is null + */ + public InterProcessMutex( + CoordinationClient client, + String coordinationNodePath, + String lockName + ) { + this( + client, + coordinationNodePath, + lockName, + InterProcessMutexSettings.newBuilder().build() + ); + } + /** + * Creates a new distributed mutex instance. + * + * @param client coordination client + * @param coordinationNodePath path to the coordination node + * @param lockName name of the lock + * @param settings configuration settings + * @throws IllegalArgumentException if any parameter is null + */ public InterProcessMutex( CoordinationClient client, String coordinationNodePath, String lockName, InterProcessMutexSettings settings ) { - lockInternals = new LockInternals(client, coordinationNodePath, lockName); - lockInternals.start(); + if (client == null || coordinationNodePath == null || lockName == null || settings == null) { + throw new IllegalArgumentException("All parameters must be non-null"); + } + + state.set(State.STARTING); + logger.debug("Initializing InterProcessMutex for lock '{}'", lockName); + + this.coordinationSession = client.createSession(coordinationNodePath); + this.sessionListenable = new ListenableContainer<>(); + this.lockInternals = new LockInternals(coordinationSession, lockName); + + coordinationSession.addStateListener(sessionState -> { + if (sessionState == CoordinationSession.State.LOST || sessionState == CoordinationSession.State.CLOSED) { + logger.error("Coordination session unexpectedly changed to {} state, marking mutex as FAILED", + sessionState); + state.set(State.FAILED); + } + sessionListenable.notifyListeners(sessionState); + }); + + sessionConnectionTask = coordinationSession.connect().thenAccept(sessionConnectStatus -> { + if (!sessionConnectStatus.isSuccess()) { + state.set(State.FAILED); + logger.error("Failed to establish coordination session for lock '{}'", lockName); + } else { + state.set(State.STARTED); + logger.info("Successfully established session for lock '{}'", lockName); + } + }); + if (settings.isWaitConnection()) { - lockInternals.getConnectedCoordinationSession(); + try { + logger.debug("Waiting for session connection to complete..."); + sessionConnectionTask.get(); + logger.debug("Session connection completed"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while waiting for session connection for lock '{}'", lockName, e); + throw new LockInitializationException("Interrupted while initializing lock '" + lockName + "'", e, + lockName); + } catch (ExecutionException e) { + logger.error("Failed to initialize lock '{}' due to execution error", lockName, e); + throw new LockInitializationException("Failed to initialize lock '" + lockName + "'", e.getCause(), + lockName); + } } } @Override public void acquire() throws Exception { + checkState(); + logger.debug("Attempting to acquire lock..."); lockInternals.tryAcquire( null, true, null ); + logger.debug("Lock acquired successfully"); } @Override public boolean acquire(Duration waitDuration) throws Exception { + checkState(); + logger.debug("Attempting to acquire lock with timeout {}...", waitDuration); Instant deadline = Instant.now().plus(waitDuration); - return lockInternals.tryAcquire( + boolean acquired = lockInternals.tryAcquire( deadline, true, null ) != null; + logger.debug("Lock acquisition {}successful", acquired ? "" : "un"); + return acquired; } @Override public boolean release() throws InterruptedException { - return lockInternals.release(); + checkState(); + logger.debug("Attempting to release lock..."); + boolean released = lockInternals.release(); + if (released) { + logger.debug("Lock released successfully"); + } else { + logger.debug("No lock to release"); + } + return released; } @Override public boolean isAcquiredInThisProcess() { - return lockInternals.isAcquired(); + try { + checkState(); + boolean acquired = lockInternals.isAcquired(); + logger.trace("Lock acquisition check: {}", acquired); + return acquired; + } catch (LockException e) { + logger.debug("Lock state check failed", e); + return false; + } } @Override - public void close() { - lockInternals.close(); + public Listenable getSessionListenable() { + return sessionListenable; } @Override - public Listenable getSessionListenable() { - return lockInternals.getSessionListenable(); + public void close() { + logger.debug("Closing InterProcessMutex..."); + state.set(State.CLOSED); + try { + lockInternals.close(); + } catch (Exception e) { + logger.warn("Error while closing lock internals", e); + } + logger.info("InterProcessMutex closed"); + } + + private void checkState() throws LockStateException { + State currentState = state.get(); + if (currentState == State.FAILED) { + throw new LockStateException("Lock '" + lockInternals.getLockName() + "' is in FAILED state", + lockInternals.getLockName()); + } + if (currentState == State.CLOSED) { + throw new LockStateException("Lock '" + lockInternals.getLockName() + "' is already closed", + lockInternals.getLockName()); + } + if (currentState != State.STARTED) { + throw new LockStateException("Lock '" + lockInternals.getLockName() + "' is not ready (current state: " + + currentState + ")", lockInternals.getLockName()); + } } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java index 5de77842c..e2a419010 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutexSettings.java @@ -1,6 +1,5 @@ package tech.ydb.coordination.recipes.locks; -// TODO: More settings public class InterProcessMutexSettings { private final boolean waitConnection; diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java deleted file mode 100644 index a0c3c7951..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAcquireFailedException.java +++ /dev/null @@ -1,28 +0,0 @@ -package tech.ydb.coordination.recipes.locks; - -public class LockAcquireFailedException extends RuntimeException { - private final String coordinationNodePath; - private final String semaphoreName; - - public LockAcquireFailedException(String message, String coordinationNodePath, String semaphoreName) { - super("Failed to acquire semaphore=" + semaphoreName + - ", on coordination node=" + coordinationNodePath + - ": '" + message + "'"); - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = semaphoreName; - } - - public LockAcquireFailedException(String coordinationNodePath, String semaphoreName) { - super("Failed to acquire semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath); - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = semaphoreName; - } - - public String getCoordinationNodePath() { - return coordinationNodePath; - } - - public String getSemaphoreName() { - return semaphoreName; - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java deleted file mode 100644 index 8b127d062..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockAlreadyAcquiredException.java +++ /dev/null @@ -1,11 +0,0 @@ -package tech.ydb.coordination.recipes.locks; - -public class LockAlreadyAcquiredException extends LockAcquireFailedException { - public LockAlreadyAcquiredException(String coordinationNodePath, String semaphoreName) { - super( - "Lock=" + semaphoreName + " on path=" + coordinationNodePath + " is already acquired", - coordinationNodePath, - semaphoreName - ); - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java index b66e060c5..5f0296180 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockInternals.java @@ -12,29 +12,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.SemaphoreLease; -import tech.ydb.coordination.recipes.util.ListenableContainer; -import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.SessionListenableProvider; +import tech.ydb.coordination.recipes.locks.exception.LockAcquireFailedException; +import tech.ydb.coordination.recipes.locks.exception.LockAlreadyAcquiredException; +import tech.ydb.coordination.recipes.locks.exception.LockReleaseFailedException; import tech.ydb.core.Result; import tech.ydb.core.Status; import tech.ydb.core.StatusCode; -public class LockInternals implements Closeable, SessionListenableProvider { +public class LockInternals implements Closeable { private static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(30); private static final Logger logger = LoggerFactory.getLogger(LockInternals.class); private final boolean persistent; private final long maxPersistentLease; - private final String coordinationNodePath; - private final String semaphoreName; - private final CoordinationSession session; - private final ListenableContainer sessionListenable = new ListenableContainer(); + private final String lockName; + private final CoordinationSession coordinationSession; - private CompletableFuture sessionConnectionTask = null; private volatile LeaseData leaseData = null; public static class LeaseData { @@ -71,94 +67,53 @@ public String toString() { } public LockInternals( - CoordinationSession session, - String lockName, - long maxPersistentLease - ) { - this.persistent = false; - this.maxPersistentLease = -1; - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = lockName; - this.session = client.createSession(coordinationNodePath); - } - - public LockInternals( - CoordinationClient client, - String coordinationNodePath, + CoordinationSession coordinationSession, String lockName ) { - this.persistent = false; - this.maxPersistentLease = -1; - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = lockName; - this.session = client.createSession(coordinationNodePath); + this(coordinationSession, lockName, null); } public LockInternals( - long maxPersistentLease, - CoordinationClient client, - String coordinationNodePath, - String lockName + CoordinationSession coordinationSession, + String lockName, + @Nullable Long maxPersistentLease ) { - this.persistent = true; - this.maxPersistentLease = maxPersistentLease; - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = lockName; - this.session = client.createSession(coordinationNodePath); - } - - public void start() { - this.sessionConnectionTask = session.connect().thenApply(status -> { - logger.debug("Session connection status: {}", status); - return status; - }); + if (maxPersistentLease == null) { + this.persistent = false; + this.maxPersistentLease = -1; + } else { + this.persistent = true; + this.maxPersistentLease = maxPersistentLease; + } - Consumer listener = state -> { - switch (state) { - case RECONNECTED: { - logger.debug("Session RECONNECTED"); - reconnect(); - break; + this.lockName = lockName; + this.coordinationSession = coordinationSession; + this.coordinationSession.addStateListener(new Consumer() { + @Override + public void accept(CoordinationSession.State state) { + switch (state) { + case RECONNECTED: + onReconnect(); + break; + case CLOSED: + case LOST: + leaseData = null; + break; + default: } - case CLOSED: { - logger.debug("Session CLOSED, releasing lock"); - leaseData = null; - break; - } - case LOST: { - logger.debug("Session LOST, releasing lock"); - leaseData = null; - break; - } - default: } - sessionListenable.notifyListeners(state); - }; - - session.addStateListener(listener); - } - - private CoordinationSession connectedSession() { - if (sessionConnectionTask == null) { - throw new IllegalStateException("Not started yet"); - } - try { - sessionConnectionTask.get().expectSuccess("Unable to connect to session on: " + coordinationNodePath); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - return session; + }); } - private void reconnect() { + private void onReconnect() { LeaseData currentLeaseData = leaseData; long oldId = currentLeaseData.getLeaseSessionId(); - long newId = session.getId(); + long newId = coordinationSession.getId(); if (oldId != newId) { logger.warn( "Current session with new id: {} lost lease after reconnection on semaphore: {}", newId, - semaphoreName + lockName ); leaseData = null; } else { @@ -167,26 +122,25 @@ private void reconnect() { } public synchronized boolean release() throws LockReleaseFailedException, InterruptedException { - logger.debug("Trying to release semaphore '{}'", semaphoreName); + logger.debug("Trying to release semaphore '{}'", lockName); - if (!connectedSession().getState().isActive()) { + if (!coordinationSession.getState().isActive()) { throw new LockReleaseFailedException( "Coordination session is inactive", - coordinationNodePath, - semaphoreName + lockName ); } LeaseData localLeaseData = leaseData; if (localLeaseData == null) { - logger.debug("Semaphore '{}' already released", semaphoreName); + logger.debug("Semaphore '{}' already released", lockName); return false; } try { localLeaseData.getProcessLease().release().get(); leaseData = null; - logger.debug("Successfully released semaphore '{}'", semaphoreName); + logger.debug("Successfully released semaphore '{}'", lockName); return true; } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -194,9 +148,8 @@ public synchronized boolean release() throws LockReleaseFailedException, Interru } catch (ExecutionException e) { throw new LockReleaseFailedException( "Failed to release lock: " + e.getCause().getMessage(), - coordinationNodePath, - semaphoreName, - e.getCause() + e.getCause(), + lockName ); } } @@ -206,21 +159,21 @@ public synchronized LeaseData tryAcquire( boolean exclusive, byte @Nullable [] data ) throws Exception { - logger.debug("Trying to acquire lock: {} with deadline: {}, exclusive: {}", semaphoreName, deadline, exclusive); + logger.debug("Trying to acquire lock: {} with deadline: {}, exclusive: {}", lockName, deadline, exclusive); if (leaseData != null) { - throw new LockAlreadyAcquiredException(coordinationNodePath, semaphoreName); + throw new LockAlreadyAcquiredException(lockName); } Optional lease = tryBlockingLock(deadline, exclusive, data); if (lease.isPresent()) { LeaseData localLeaseData = new LeaseData(lease.get(), exclusive, 1); leaseData = localLeaseData; - logger.debug("Successfully acquired lock: {}", semaphoreName); + logger.debug("Successfully acquired lock: {}", lockName); return localLeaseData; } - logger.debug("Unable to acquire lock: {}", semaphoreName); + logger.debug("Unable to acquire lock: {}", lockName); return null; } @@ -230,8 +183,6 @@ private Optional tryBlockingLock( byte @Nullable [] data ) throws Exception { int retryCount = 0; - CoordinationSession coordinationSession = connectedSession(); - while (coordinationSession.getState().isActive() && (deadline == null || Instant.now().isBefore(deadline))) { retryCount++; @@ -242,9 +193,7 @@ private Optional tryBlockingLock( timeout = Duration.between(Instant.now(), deadline); } - CompletableFuture> acquireTask = acquire( - exclusive, data, coordinationSession, timeout - ); + CompletableFuture> acquireTask = acquireCall(exclusive, data, timeout); Result leaseResult; try { @@ -266,18 +215,18 @@ private Optional tryBlockingLock( logger.debug("Lease result status: {}", status); if (status.isSuccess()) { - logger.debug("Successfully acquired the lock '{}'", semaphoreName); + logger.debug("Successfully acquired the lock '{}'", lockName); return Optional.of(leaseResult.getValue()); } if (status.getCode() == StatusCode.TIMEOUT) { - logger.debug("Trying to acquire semaphore {} again, retries: {}", semaphoreName, retryCount); + logger.debug("Trying to acquire semaphore {} again, retries: {}", lockName, retryCount); continue; } if (!status.getCode().isRetryable(true)) { - logger.debug("Unable to retry acquiring semaphore '{}'", semaphoreName); - throw new LockAcquireFailedException(coordinationNodePath, semaphoreName); + logger.debug("Unable to retry acquiring semaphore '{}'", lockName); + throw new LockAcquireFailedException(lockName); } } @@ -285,40 +234,27 @@ private Optional tryBlockingLock( return Optional.empty(); } - throw new LockAcquireFailedException(coordinationNodePath, semaphoreName); + throw new LockAcquireFailedException(lockName); } - private CompletableFuture> acquire( + private CompletableFuture> acquireCall( boolean exclusive, byte[] data, - CoordinationSession coordinationSession, Duration timeout ) { if (!persistent) { - return coordinationSession.acquireEphemeralSemaphore(semaphoreName, exclusive, data, timeout); + return coordinationSession.acquireEphemeralSemaphore(lockName, exclusive, data, timeout); } if (exclusive) { - return coordinationSession.acquireSemaphore(semaphoreName, maxPersistentLease, data, timeout); + return coordinationSession.acquireSemaphore(lockName, maxPersistentLease, data, timeout); } - return coordinationSession.acquireSemaphore(semaphoreName, 1, data, timeout); - } - - public String getCoordinationNodePath() { - return coordinationNodePath; - } - - public String getSemaphoreName() { - return semaphoreName; - } - - public CoordinationSession getCoordinationSession() { - return session; + return coordinationSession.acquireSemaphore(lockName, 1, data, timeout); } - public CoordinationSession getConnectedCoordinationSession() { - return connectedSession(); + public String getLockName() { + return lockName; } public @Nullable LeaseData getLeaseData() { @@ -333,18 +269,12 @@ public boolean isPersistent() { return persistent; } - @Override - public Listenable getSessionListenable() { - return sessionListenable; - } - @Override public void close() { try { release(); - } catch (Exception ignored) { + } catch (Exception exception) { + logger.error("Exception during closing release", exception); } - - session.close(); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java deleted file mode 100644 index 413fc6bbd..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/LockReleaseFailedException.java +++ /dev/null @@ -1,38 +0,0 @@ -package tech.ydb.coordination.recipes.locks; - -public class LockReleaseFailedException extends RuntimeException { - private final String coordinationNodePath; - private final String semaphoreName; - - public LockReleaseFailedException(String message, String coordinationNodePath, String semaphoreName, Throwable e) { - super( - "Failed to release semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath + - ": '" + message + "'", e - ); - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = semaphoreName; - } - - public LockReleaseFailedException(String message, String coordinationNodePath, String semaphoreName) { - super( - "Failed to release semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath + - ": '" + message + "'" - ); - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = semaphoreName; - } - - public LockReleaseFailedException(String coordinationNodePath, String semaphoreName) { - super("Failed to release semaphore=" + semaphoreName + ", on coordination node=" + coordinationNodePath); - this.coordinationNodePath = coordinationNodePath; - this.semaphoreName = semaphoreName; - } - - public String getCoordinationNodePath() { - return coordinationNodePath; - } - - public String getSemaphoreName() { - return semaphoreName; - } -} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java index d81e7edf9..76649288d 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java @@ -4,6 +4,7 @@ import java.time.Duration; import java.time.Instant; import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -11,13 +12,12 @@ import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.recipes.util.Listenable; +import tech.ydb.coordination.recipes.util.ListenableContainer; -// TODO: add documentation and logs +// TODO: state management + (документация, логи, рекомендации) public class ReadWriteInterProcessLock implements Closeable { private static final Logger logger = LoggerFactory.getLogger(ReadWriteInterProcessLock.class); - private final LockInternals readLockInternals; - private final LockInternals writeLockInternals; private final InternalLock readLock; private final InternalLock writeLock; @@ -26,21 +26,22 @@ public ReadWriteInterProcessLock( String coordinationNodePath, String lockName ) { - this.readLockInternals = new LockInternals( + this( client, coordinationNodePath, - lockName + lockName, + ReadWriteInterProcessLockSettings.newBuilder().build() ); - readLockInternals.start(); - this.readLock = new InternalLock(readLockInternals, false); + } - this.writeLockInternals = new LockInternals( - client, - coordinationNodePath, - lockName - ); - writeLockInternals.start(); - this.writeLock = new InternalLock(writeLockInternals, true); + public ReadWriteInterProcessLock( + CoordinationClient client, + String coordinationNodePath, + String lockName, + ReadWriteInterProcessLockSettings settings + ) { + this.readLock = new InternalLock(client, coordinationNodePath, lockName, false); + this.writeLock = new InternalLock(client, coordinationNodePath, lockName, true); } public InterProcessLock writeLock() { @@ -52,12 +53,44 @@ public InterProcessLock readLock() { } private static class InternalLock implements InterProcessLock { - private final LockInternals lockInternals; private final boolean isExclusive; - private InternalLock(LockInternals lockInternals, boolean isExclusive) { - this.lockInternals = lockInternals; + private final AtomicReference state = new AtomicReference<>(State.INITIAL); + private final CoordinationSession coordinationSession; + private final LockInternals lockInternals; + private final ListenableContainer sessionListenable; + + private enum State { + INITIAL, + STARTING, + STARTED, + FAILED, + CLOSED + } + + public InternalLock( + CoordinationClient client, + String coordinationNodePath, + String lockName, + boolean isExclusive + ) { this.isExclusive = isExclusive; + + this.coordinationSession = client.createSession(coordinationNodePath); + this.lockInternals = new LockInternals( + coordinationSession, + lockName + ); + this.sessionListenable = new ListenableContainer<>(); + coordinationSession.addStateListener(sessionListenable::notifyListeners); + // TODO: add settings to block + coordinationSession.connect().thenAccept(status -> { + status.expectSuccess("Unable to establish session"); + }); + } + + private void start() { + ; } @Override @@ -93,14 +126,18 @@ public boolean isAcquiredInThisProcess() { @Override public Listenable getSessionListenable() { - return lockInternals.getSessionListenable(); + return sessionListenable; + } + + private void close() { + lockInternals.close(); } } @Override public void close() { - readLockInternals.close(); - writeLockInternals.close(); + readLock.close(); + writeLock.close(); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockSettings.java new file mode 100644 index 000000000..aef384c40 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockSettings.java @@ -0,0 +1,31 @@ +package tech.ydb.coordination.recipes.locks; + +public class ReadWriteInterProcessLockSettings { + private final boolean waitConnection; + + public ReadWriteInterProcessLockSettings(Builder builder) { + this.waitConnection = builder.waitConnection; + } + + public boolean isWaitConnection() { + return waitConnection; + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private boolean waitConnection = false; + + public Builder withWaitConnection(boolean waitConnection) { + this.waitConnection = waitConnection; + return this; + } + + public ReadWriteInterProcessLockSettings build() { + return new ReadWriteInterProcessLockSettings(this); + } + } +} + diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAcquireFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAcquireFailedException.java new file mode 100644 index 000000000..ac41a8505 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAcquireFailedException.java @@ -0,0 +1,18 @@ +package tech.ydb.coordination.recipes.locks.exception; + +/** + * Thrown when lock acquisition fails (excluding cases when lock is already acquired) + */ +public class LockAcquireFailedException extends LockException { + public LockAcquireFailedException(String lockName) { + super("Failed to acquire lock '" + lockName + "'", lockName); + } + + public LockAcquireFailedException(String message, String lockName) { + super(message, lockName); + } + + public LockAcquireFailedException(String message, Throwable cause, String lockName) { + super(message, cause, lockName); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAlreadyAcquiredException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAlreadyAcquiredException.java new file mode 100644 index 000000000..f72d88745 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockAlreadyAcquiredException.java @@ -0,0 +1,18 @@ +package tech.ydb.coordination.recipes.locks.exception; + +/** + * Thrown when attempting to acquire a lock that is already held by current process + */ +public class LockAlreadyAcquiredException extends LockException { + public LockAlreadyAcquiredException(String lockName) { + super("Lock '" + lockName + "' is already acquired by this process", lockName); + } + + public LockAlreadyAcquiredException(String message, String lockName) { + super(message, lockName); + } + + public LockAlreadyAcquiredException(String message, Throwable cause, String lockName) { + super(message, cause, lockName); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockException.java new file mode 100644 index 000000000..ebdd9bbcf --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockException.java @@ -0,0 +1,23 @@ +package tech.ydb.coordination.recipes.locks.exception; + +public class LockException extends RuntimeException { + protected final String lockName; + + public LockException(String lockName) { + this.lockName = lockName; + } + + public LockException(String message, String lockName) { + super(message); + this.lockName = lockName; + } + + public LockException(String message, Throwable cause, String lockName) { + super(message, cause); + this.lockName = lockName; + } + + public String getLockName() { + return lockName; + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockInitializationException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockInitializationException.java new file mode 100644 index 000000000..583b84ea7 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockInitializationException.java @@ -0,0 +1,18 @@ +package tech.ydb.coordination.recipes.locks.exception; + +/** + * Thrown when lock initialization fails + */ +public class LockInitializationException extends LockException { + public LockInitializationException(String lockName) { + super("Failed to initialize lock '" + lockName + "'", lockName); + } + + public LockInitializationException(String message, String lockName) { + super(message, lockName); + } + + public LockInitializationException(String message, Throwable cause, String lockName) { + super(message, cause, lockName); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockReleaseFailedException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockReleaseFailedException.java new file mode 100644 index 000000000..fe3e04287 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockReleaseFailedException.java @@ -0,0 +1,18 @@ +package tech.ydb.coordination.recipes.locks.exception; + +/** + * Thrown when lock release operation fails + */ +public class LockReleaseFailedException extends LockException { + public LockReleaseFailedException(String lockName) { + super("Failed to release lock '" + lockName + "'", lockName); + } + + public LockReleaseFailedException(String message, String lockName) { + super(message, lockName); + } + + public LockReleaseFailedException(String message, Throwable cause, String lockName) { + super(message, cause, lockName); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockStateException.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockStateException.java new file mode 100644 index 000000000..4e81d7da8 --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/exception/LockStateException.java @@ -0,0 +1,19 @@ +package tech.ydb.coordination.recipes.locks.exception; + +/** + * Thrown when lock operation is attempted in invalid state + */ +public class LockStateException extends LockException { + public LockStateException(String lockName) { + super("Invalid state for lock operation on lock '" + lockName + "'", lockName); + } + + public LockStateException(String message, String lockName) { + super(message, lockName); + } + + public LockStateException(String message, Throwable cause, String lockName) { + super(message, cause, lockName); + } +} + diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java index 8aa08ae89..9143d7864 100644 --- a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java @@ -2,10 +2,8 @@ import java.time.Duration; import java.util.HashSet; -import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Consumer; import org.junit.Assert; @@ -32,7 +30,6 @@ public class CoordinationSessionBaseMockedTest { private final CoordinationClient client = Mockito.mock(CoordinationClient.class); private final CoordinationSession coordinationSession = Mockito.mock(CoordinationSession.class); private final SessionMock sessionMock = new SessionMock(); - private final SessionStateAssert sessionStateAssert = new SessionStateAssert(); @Before public void beforeEach() { @@ -69,10 +66,6 @@ public CoordinationSession getCoordinationSession() { return coordinationSession; } - public SessionStateAssert getSessionStateAssert() { - return sessionStateAssert; - } - protected Answer> successConnect() { return (InvocationOnMock iom) -> { logger.debug("Successful session connect"); @@ -154,27 +147,6 @@ protected SessionMock getSessionMock() { return sessionMock; } - protected class SessionStateAssert implements Consumer { - private final Queue queue = new ConcurrentLinkedQueue<>(); - - public SessionStateAssert next(CoordinationSession.State state) { - queue.add(state); - return this; - } - - @Override - public void accept(CoordinationSession.State state) { - logger.debug("Next state: {}", state); - Assert.assertFalse(queue.isEmpty()); - CoordinationSession.State lastState = queue.poll(); - Assert.assertEquals(state, lastState); - } - - public void finished() { - Assert.assertTrue(queue.isEmpty()); - } - } - protected class LeaseMock implements SemaphoreLease { private final SessionMock sessionMock; private final String name; diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java deleted file mode 100644 index e858701c9..000000000 --- a/coordination/src/test/java/tech/ydb/coordination/CoordinationStreamBaseMockedTest.java +++ /dev/null @@ -1,223 +0,0 @@ -package tech.ydb.coordination; - -import org.jetbrains.annotations.NotNull; -import org.junit.Assert; -import org.junit.Before; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import tech.ydb.core.Status; -import tech.ydb.core.StatusCode; -import tech.ydb.core.grpc.GrpcReadWriteStream; -import tech.ydb.core.grpc.GrpcTransport; -import tech.ydb.proto.StatusCodesProtos; -import tech.ydb.proto.coordination.SessionRequest; -import tech.ydb.proto.coordination.SessionResponse; -import tech.ydb.proto.coordination.v1.CoordinationServiceGrpc; - -import java.util.ArrayList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.Executor; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; - -public class CoordinationStreamBaseMockedTest { - private static final Logger logger = LoggerFactory.getLogger(CoordinationStreamBaseMockedTest.class); - - private final ScheduledExecutorService scheduler = Mockito.mock(ScheduledExecutorService.class); - private final GrpcTransport transport = Mockito.mock(GrpcTransport.class); - private final ScheduledFuture emptyFuture = Mockito.mock(ScheduledFuture.class); - private final SchedulerAssert schedulerHelper = new SchedulerAssert(); - - protected final CoordinationClient client = CoordinationClient.newClient(transport); - - @Before - public void beforeEach() { - Mockito.when(transport.getScheduler()).thenReturn(scheduler); - - Mockito.when(scheduler.schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any())) - .thenAnswer((InvocationOnMock iom) -> { - logger.debug("mock scheduled task"); - schedulerHelper.tasks.add(iom.getArgument(0, Runnable.class)); - return emptyFuture; - }); - } - - protected SchedulerAssert getScheduler() { - return schedulerHelper; - } - - protected StreamMock mockStream() { - StreamMock streamMock = new StreamMock(); - - GrpcReadWriteStream readWriteStream = Mockito.mock(GrpcReadWriteStream.class); - - Mockito.when(readWriteStream.start(Mockito.any())).thenAnswer( - (InvocationOnMock iom) -> { - streamMock.setObserver(iom.getArgument(0)); - return streamMock.streamFuture; - } - ).thenThrow(new RuntimeException("Unexpected second start call")); - - Mockito.doAnswer((Answer) (InvocationOnMock iom) -> { - streamMock.sent.add(iom.getArgument(0, SessionRequest.class)); - return null; - }).when(readWriteStream).sendNext(Mockito.any()); - - Mockito.when(transport.readWriteStreamCall(Mockito.eq(CoordinationServiceGrpc.getSessionMethod()), Mockito.any())) - .thenReturn(readWriteStream); - return streamMock; - } - - protected static class SchedulerAssert implements Executor { - private final Queue tasks = new ConcurrentLinkedQueue<>(); - - @Override - public void execute(@NotNull Runnable command) { - logger.debug("scheduling command: " + command); - tasks.add(command); - } - - public SchedulerAssert hasNoTasks() { - Assert.assertTrue(tasks.isEmpty()); - return this; - } - - public SchedulerAssert hasTasks(int count) { - Assert.assertEquals(count, tasks.size()); - return this; - } - - public SchedulerAssert executeNextTasks(int count) { - Assert.assertTrue(count <= tasks.size()); - - CompletableFuture.runAsync(() -> { - logger.debug("execute {} scheduled tasks", count); - for (int idx = 0; idx < count; idx++) { - tasks.poll().run(); - } - }).join(); - return this; - } - } - - protected class StreamMock { - private final CompletableFuture streamFuture; - private final List sent = new ArrayList<>(); - private volatile int sentIdx = 0; - - private volatile GrpcReadWriteStream.Observer observer = null; - - public StreamMock() { - streamFuture = new CompletableFuture<>(); - } - - public void setObserver(GrpcReadWriteStream.Observer observer) { - this.observer = observer; - } - - public void complete(StatusCode statusCode) { - streamFuture.complete(Status.of(statusCode)); - } - - public void complete(Status status) { - streamFuture.complete(status); - } - - public void complete(Throwable th) { - streamFuture.completeExceptionally(th); - } - - public void hasNoNewMessages() { - Assert.assertTrue(sentIdx >= sent.size()); - } - - public Checker nextMsg() { - Assert.assertTrue(sentIdx < sent.size()); - return new Checker(sent.get(sentIdx++)); - } - - public void responseSemaphoreAlreadyExists() { - SessionResponse msg = SessionResponse.newBuilder() - .setAcquireSemaphoreResult( - SessionResponse.AcquireSemaphoreResult.newBuilder().setStatus( - StatusCodesProtos.StatusIds.StatusCode.ALREADY_EXISTS - ) - ) - .build(); - response(msg); - } - - public void responseSessionStarted(long sessionId) { - SessionResponse msg = SessionResponse.newBuilder() - .setSessionStarted( - SessionResponse.SessionStarted.newBuilder() - .setSessionId(sessionId) - .build() - ) - .build(); - response(msg); - } - - public void responseAcquiredSuccessfully(long requestId) { - SessionResponse msg = SessionResponse.newBuilder() - .setAcquireSemaphoreResult( - SessionResponse.AcquireSemaphoreResult.newBuilder() - .setReqId(requestId) - .setAcquired(true) - .setStatus(StatusCodesProtos.StatusIds.StatusCode.SUCCESS) - ) - .build(); - response(msg); - } - - private void response(SessionResponse msg) { - Assert.assertNotNull(observer); - observer.onNext(msg); - } - - } - - protected class Checker { - private final SessionRequest msg; - - public Checker(SessionRequest msg) { - this.msg = msg; - } - - public SessionRequest get() { - return msg; - } - - public Checker isAcquireSemaphore() { - Assert.assertTrue("next msg must be acquire semaphore", msg.hasAcquireSemaphore()); - return this; - } - - public Checker isEphemeralSemaphore() { - Assert.assertTrue("next msg must be acquire ephemeral semaphore", msg.getAcquireSemaphore().getEphemeral()); - return this; - } - - public Checker hasSemaphoreName(String semaphoreName) { - Assert.assertEquals("invalid semaphore name", semaphoreName, msg.getAcquireSemaphore().getName()); - return this; - } - - public Checker isSessionStart() { - Assert.assertTrue("next msg must be session start", msg.hasSessionStart()); - return this; - } - - public Checker hasPath(String coordinationNodePath) { - Assert.assertEquals("invalid coordination node path", coordinationNodePath, msg.getSessionStart().getPath()); - return this; - } - } -} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java index 097c746f9..bea966048 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java @@ -1,9 +1,9 @@ package tech.ydb.coordination.recipes.election; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.List; import java.util.Optional; -import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BooleanSupplier; @@ -41,22 +41,27 @@ private LeaderElection getLeaderElector( String testName, LeaderElectionListener leaderElectionListener ) { - return getLeaderElector(testName, testName, leaderElectionListener); + return getLeaderElector( + testName, + testName, + testName.getBytes(StandardCharsets.UTF_8), + leaderElectionListener + ); } private LeaderElection getLeaderElector( String nodePath, String lockName, + byte[] data, LeaderElectionListener leaderElectionListener ) { client.createNode(nodePath).join().expectSuccess("cannot create coordination path"); LeaderElection leaderElectorImpl = new LeaderElection( client, - leaderElectionListener, nodePath, lockName, - null, - Executors.newSingleThreadExecutor() + data, + leaderElectionListener ); return leaderElectorImpl; } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java index e1bc4b1f6..bedd8eb35 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java @@ -19,6 +19,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.recipes.locks.exception.LockAlreadyAcquiredException; import tech.ydb.test.junit4.GrpcTransportRule; public class InterProcessMutexIntegrationTest { @@ -54,6 +55,7 @@ private InterProcessMutex getInterProcessMutex(String nodePath, String lockName) nodePath, lockName, InterProcessMutexSettings.newBuilder() + .withWaitConnection(true) .build() ); return lock; diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java index d8e8b7240..a35cadcf0 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InternalLockMockedTest.java @@ -10,8 +10,10 @@ import org.junit.Assert; import org.junit.Test; -import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.CoordinationSessionBaseMockedTest; +import tech.ydb.coordination.recipes.locks.exception.LockAcquireFailedException; +import tech.ydb.coordination.recipes.locks.exception.LockAlreadyAcquiredException; +import tech.ydb.coordination.recipes.locks.exception.LockReleaseFailedException; import tech.ydb.core.StatusCode; import static org.mockito.ArgumentMatchers.any; @@ -82,11 +84,9 @@ public void acquire_RespondedStatusSuccess_AcquiresLock() throws Exception { .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); LeaseMock lease = lease("lock_name"); @@ -114,11 +114,9 @@ public void acquire_WithCustomData_PropagatesToSession() throws Exception { sessionMock.connect().then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); LeaseMock lease = lease("lock_name"); @@ -146,13 +144,9 @@ public void acquire_LostSession_ThrowsLockAcquireFailedException() { .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - verify(getCoordinationSession()) - .connect(); sessionMock.lost(); @@ -170,14 +164,9 @@ public void acquire_SessionLostDuringBlock_ThrowsLockAcquireFailedException() { .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - verify(getCoordinationSession()) - .connect(); - sessionMock.connected(); sessionMock.acquireEphemeralSemaphore() @@ -206,21 +195,11 @@ public void acquire_RespondedSuccessStatusThenLostSession_ReleasedLock() throws .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - lock.getSessionListenable().addListener( - getSessionStateAssert() - .next(CoordinationSession.State.CONNECTING) - .next(CoordinationSession.State.CONNECTED) - .next(CoordinationSession.State.LOST) - ); sessionMock.connecting(); - verify(getCoordinationSession()) - .connect(); sessionMock.connected(); sessionMock.acquireEphemeralSemaphore() @@ -240,7 +219,6 @@ public void acquire_RespondedSuccessStatusThenLostSession_ReleasedLock() throws sessionMock.lost(); Assert.assertFalse(lock.isAcquired()); - getSessionStateAssert().finished(); } @Test @@ -250,14 +228,9 @@ public void acquire_RespondedRetryableStatusThenSuccessStatus_AcquiredLock() thr .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - verify(getCoordinationSession()) - .connect(); - sessionMock.connected(); sessionMock.acquireEphemeralSemaphore() @@ -290,14 +263,9 @@ public void acquire_RespondedNonRetryableStatus_ThrowsLockAcquireFailedException .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - verify(getCoordinationSession()) - .connect(); - sessionMock.connected(); sessionMock.acquireEphemeralSemaphore() @@ -326,14 +294,9 @@ public void acquire_BlockingInterrupted_ThrowsInterruptedException() throws Inte .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - verify(getCoordinationSession()) - .connect(); - sessionMock.connected(); sessionMock.acquireEphemeralSemaphore() @@ -371,14 +334,9 @@ public void acquire_CallAcquireTwice_ThrowsLockAlreadyAcquiredException() throws .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); - verify(getCoordinationSession()) - .connect(); - sessionMock.connected(); sessionMock.acquireEphemeralSemaphore() @@ -414,11 +372,9 @@ public void acquireWithTimeout_RespondedSuccessStatus_AcquiredLock() throws Exce .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); LeaseMock lease = lease("lock_name"); @@ -448,11 +404,9 @@ public void acquireWithTimeout_ResponseTimeout_ReturnsFalse() throws Exception { .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); LeaseMock lease = lease("lock_name"); @@ -481,11 +435,9 @@ public void release_RespondedStatusSuccess_ReleasedLock() throws Exception { .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); LeaseMock lease = lease("lock_name"); @@ -510,11 +462,9 @@ public void release_NoCurrentLease_ReturnedFalse() throws Exception { .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); Assert.assertFalse(lock.release()); @@ -528,11 +478,9 @@ public void release_AlreadySessionLost_ThrowsLockReleaseFailedException() throws .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.lost(); @@ -547,11 +495,9 @@ public void release_DuringSessionLost_ThrowsLockReleaseFailedException() throws .then(successConnect()); LockInternals lock = new LockInternals( - getClient(), - "/node/path", + getCoordinationSession(), "lock_name" ); - lock.start(); sessionMock.connected(); LeaseMock lease = lease("lock_name") From 15053da56d9de1ed64e26fe3add3a0b3c1bce8fc Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Tue, 6 May 2025 00:30:29 +0300 Subject: [PATCH 05/10] finished semaphore observer --- .../recipes/election/LeaderElection.java | 136 ++++++++++---- .../election/LeaderElectionSettings.java | 17 +- .../recipes/group/GroupMembershipImpl.java | 21 ++- .../group/GroupMembershipSettings.java | 52 ++++++ .../recipes/util/RetryableTask.java | 103 +++++++++++ .../recipes/util/SemaphoreObserver.java | 173 +++++++++++++++--- .../LeaderElectionIntegrationTest.java | 2 - .../group/GroupMembershipIntegrationTest.java | 4 +- .../util/SemaphoreObserverMockedTest.java | 5 +- 9 files changed, 428 insertions(+), 85 deletions(-) create mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java index 57532f632..42005917d 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -5,10 +5,14 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -16,12 +20,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import tech.ydb.common.retry.RetryPolicy; import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.description.SemaphoreDescription; import tech.ydb.coordination.recipes.locks.LockInternals; import tech.ydb.coordination.recipes.util.Listenable; import tech.ydb.coordination.recipes.util.ListenableContainer; +import tech.ydb.coordination.recipes.util.RetryableTask; import tech.ydb.coordination.recipes.util.SessionListenableProvider; import tech.ydb.coordination.recipes.util.SemaphoreObserver; import tech.ydb.coordination.settings.DescribeSemaphoreMode; @@ -29,7 +35,7 @@ import tech.ydb.core.Status; import tech.ydb.core.StatusCode; -// TODO: backoff политика + документцаия / логгирование / рекомендации по коду +// TODO: документцаия / логгирование / рекомендации по коду public class LeaderElection implements Closeable, SessionListenableProvider { private static final Logger logger = LoggerFactory.getLogger(LeaderElection.class); private static final long MAX_LEASE = 1L; @@ -38,22 +44,27 @@ public class LeaderElection implements Closeable, SessionListenableProvider { private final String coordinationNodePath; private final String electionName; private final byte[] data; + private final RetryPolicy retryPolicy; - private final ExecutorService electionExecutor; + private final ScheduledExecutorService scheduledExecutor; + private final ExecutorService blockingExecutor; private final CoordinationSession coordinationSession; private final ListenableContainer sessionListenable; private final LockInternals lock; private final SemaphoreObserver semaphoreObserver; - private AtomicReference state = new AtomicReference<>(State.CREATED); - private Future sessionConnectionTask = null; + private final CountDownLatch startingLatch = new CountDownLatch(1); + private AtomicReference state = new AtomicReference<>(State.INITIAL); + private AtomicReference> initializingTask = new AtomicReference<>(null); private Future electionTask = null; private volatile boolean autoRequeue = false; private volatile boolean isLeader = false; private enum State { - CREATED, + INITIAL, + STARTING, STARTED, + FAILED, CLOSED } @@ -87,11 +98,20 @@ public LeaderElection( this.electionName = electionName; this.data = data; this.leaderElectionListener = leaderElectionListener; - this.electionExecutor = settings.getExecutorService(); + this.scheduledExecutor = settings.getScheduledExecutor(); + this.blockingExecutor = Executors.newSingleThreadExecutor(); // TODO: thread factory + this.retryPolicy = settings.getRetryPolicy(); this.coordinationSession = client.createSession(coordinationNodePath); this.sessionListenable = new ListenableContainer<>(); - coordinationSession.addStateListener(sessionListenable::notifyListeners); + coordinationSession.addStateListener(sessionState -> { + if (sessionState == CoordinationSession.State.LOST || sessionState == CoordinationSession.State.CLOSED) { + logger.error("Coordination session unexpectedly changed to {} state, marking election as FAILED", + sessionState); + state.set(State.FAILED); + } + sessionListenable.notifyListeners(sessionState); + }); this.lock = new LockInternals( coordinationSession, electionName, @@ -102,41 +122,60 @@ public LeaderElection( electionName, WatchSemaphoreMode.WATCH_OWNERS, DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, - settings.getRetryPolicy() + settings.getRetryPolicy(), + settings.getScheduledExecutor() ); } - private CoordinationSession connectedSession() { - if (sessionConnectionTask == null) { - throw new IllegalStateException("Not started yet"); - } - try { - sessionConnectionTask.get().expectSuccess("Unable to connect to session"); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - return coordinationSession; - } - public void start() { - Preconditions.checkState(state.compareAndSet(State.CREATED, State.STARTED), "Already started or closed"); - // TODO: handle errors retries and logging? - this.sessionConnectionTask = coordinationSession.connect().thenCompose(connectionStatus -> { - connectionStatus.expectSuccess("Unable to establish session"); - return coordinationSession.createSemaphore(electionName, MAX_LEASE).thenApply(semaphoreStatus -> { - if (semaphoreStatus.isSuccess() || semaphoreStatus.getCode() == StatusCode.ALREADY_EXISTS) { - semaphoreObserver.start(); - } - semaphoreStatus.expectSuccess("Unable to create semaphore"); - return semaphoreStatus; - }); - }); + Preconditions.checkState( + state.compareAndSet(State.INITIAL, State.STARTING), + "Leader election may be started only once" + ); + + CompletableFuture connectionTask = executeWithRetry(coordinationSession::connect); + CompletableFuture semaphoreCreateTask = executeWithRetry( + () -> coordinationSession.createSemaphore(electionName, MAX_LEASE) + .thenCompose(status -> { + if (status.getCode() == StatusCode.ALREADY_EXISTS) { + return CompletableFuture.completedFuture(Status.SUCCESS); + } + return CompletableFuture.completedFuture(status); + }) + ); + + CompletableFuture initializingRetriedTask = connectionTask + .thenCompose(connectionStatus -> { + connectionStatus.expectSuccess("Unable to establish session"); + return semaphoreCreateTask; + }) + .thenApply(semaphoreStatus -> { + if (semaphoreStatus.isSuccess()) { + state.set(State.STARTED); + semaphoreObserver.start(); + startingLatch.countDown(); + } + semaphoreStatus.expectSuccess("Unable to create semaphore"); + return semaphoreStatus; + }).exceptionally(ex -> { + logger.error("Leader election initializing task failed", ex); + state.set(State.FAILED); + semaphoreObserver.close(); + startingLatch.countDown(); + return Status.of(StatusCode.CLIENT_INTERNAL_ERROR); + }); + + initializingTask.set(initializingRetriedTask); if (autoRequeue) { enqueueElection(); } } + private CompletableFuture executeWithRetry(Supplier> taskSupplier) { + return new RetryableTask("leaderElectionInitialize", taskSupplier, scheduledExecutor, retryPolicy).execute(); + } + public void autoRequeue() { autoRequeue = true; } @@ -153,9 +192,12 @@ public boolean isLeader() { * @return true if re-enqueue was successful */ public boolean requeue() { - Preconditions.checkState(state.get() == State.STARTED, "Already closed or not yet started"); + State localState = state.get(); + Preconditions.checkState( + localState == State.STARTED || localState == State.STARTING, + "Unexpected state: " + localState.name() + ); - // TODO: корректно обрабатывать если старт еще не кончился return enqueueElection(); } @@ -170,8 +212,9 @@ public synchronized boolean interruptLeadership() { } private synchronized boolean enqueueElection() { - if (!isQueued() && state.get() == State.STARTED) { - electionTask = electionExecutor.submit(new Callable() { + State localState = state.get(); + if (!isQueued() && (localState == State.STARTED || localState == State.STARTING)) { + electionTask = blockingExecutor.submit(new Callable() { @Override public Void call() throws Exception { try { @@ -192,6 +235,7 @@ private void doWork() throws Exception { isLeader = false; try { + waitStartedState(); lock.tryAcquire( null, true, @@ -226,9 +270,22 @@ private void doWork() throws Exception { } } + private void waitStartedState() throws InterruptedException { + State localState = state.get(); + if (localState == State.STARTING) { + startingLatch.await(); + localState = state.get(); + } + + if (localState == State.INITIAL || localState == State.CLOSED || localState == State.FAILED) { + throw new IllegalStateException("Unexpected state: " + localState.name()); + } + } + private synchronized void finishTask() { electionTask = null; - if (autoRequeue) { + State localState = state.get(); + if (autoRequeue && localState != State.CLOSED && localState != State.FAILED) { enqueueElection(); } } @@ -281,6 +338,7 @@ public Listenable getSessionListenable() { @Override public synchronized void close() { + // TODO: Учесть все стейты Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Already closed"); Future localTask = electionTask; @@ -289,7 +347,7 @@ public synchronized void close() { electionTask = null; } - electionExecutor.shutdown(); + blockingExecutor.shutdown(); semaphoreObserver.close(); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java index 9507c90cd..d4494a320 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionSettings.java @@ -1,8 +1,8 @@ package tech.ydb.coordination.recipes.election; import java.time.Duration; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import tech.ydb.common.retry.RetryPolicy; import tech.ydb.common.retry.RetryUntilElapsed; @@ -13,16 +13,17 @@ public class LeaderElectionSettings { DEFAULT_CONNECT_TIMEOUT.toMillis(), 250, 5 ); - private final ExecutorService executorService; + private final ScheduledExecutorService scheduledExecutor; private final RetryPolicy retryPolicy; public LeaderElectionSettings(Builder builder) { - this.executorService = builder.executorService; + this.scheduledExecutor = builder.scheduledExecutor != null ? builder.scheduledExecutor : + Executors.newSingleThreadScheduledExecutor(); this.retryPolicy = builder.retryPolicy; } - public ExecutorService getExecutorService() { - return executorService; + public ScheduledExecutorService getScheduledExecutor() { + return scheduledExecutor; } public RetryPolicy getRetryPolicy() { @@ -34,11 +35,11 @@ public static Builder newBuilder() { } public static class Builder { - private ExecutorService executorService = Executors.newSingleThreadExecutor(); + private ScheduledExecutorService scheduledExecutor; private RetryPolicy retryPolicy = DEFAULT_RETRY_POLICY; - public Builder withExecutorService(ExecutorService executorService) { - this.executorService = executorService; + public Builder withScheduledExecutor(ScheduledExecutorService executorService) { + this.scheduledExecutor = executorService; return this; } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java index db584407c..342d25b85 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java @@ -31,16 +31,30 @@ public class GroupMembershipImpl implements GroupMembership { private final ListenableContainer sessionStateListenable; private final ListenableContainer> groupMembersListenable; + public GroupMembershipImpl( + CoordinationClient coordinationClient, + String coordinationNodePath, + String groupName + ) { + this( + coordinationClient, + coordinationNodePath, + groupName, + GroupMembershipSettings.newBuilder() + .build() + ); + } + public GroupMembershipImpl( CoordinationClient coordinationClient, String coordinationNodePath, String groupName, - RetryPolicy retryPolicy + GroupMembershipSettings settings ) { this.coordinationClient = coordinationClient; this.coordinationNodePath = coordinationNodePath; this.groupName = groupName; - this.retryPolicy = retryPolicy; + this.retryPolicy = settings.getRetryPolicy(); this.session = coordinationClient.createSession( coordinationNodePath, @@ -62,7 +76,8 @@ public GroupMembershipImpl( groupName, WatchSemaphoreMode.WATCH_OWNERS, DescribeSemaphoreMode.WITH_OWNERS, - retryPolicy + settings.getRetryPolicy(), + settings.getScheduledExecutor() ); this.groupMembersListenable = new ListenableContainer<>(); semaphoreObserver.getWatchDataListenable().addListener(description -> { diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java index 23d8dbe2d..c5c932200 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipSettings.java @@ -1,4 +1,56 @@ package tech.ydb.coordination.recipes.group; +import java.time.Duration; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.common.retry.RetryUntilElapsed; + public class GroupMembershipSettings { + public static final Duration DEFAULT_CONNECT_TIMEOUT = Duration.ofSeconds(5); + public static final RetryUntilElapsed DEFAULT_RETRY_POLICY = new RetryUntilElapsed( + DEFAULT_CONNECT_TIMEOUT.toMillis(), 250, 5 + ); + + private final ScheduledExecutorService scheduledExecutor; + private final RetryPolicy retryPolicy; + + public GroupMembershipSettings(Builder builder) { + this.scheduledExecutor = builder.scheduledExecutor != null ? builder.scheduledExecutor : + Executors.newSingleThreadScheduledExecutor(); + this.retryPolicy = builder.retryPolicy; + } + + public RetryPolicy getRetryPolicy() { + return retryPolicy; + } + + public ScheduledExecutorService getScheduledExecutor() { + return scheduledExecutor; + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private RetryPolicy retryPolicy = DEFAULT_RETRY_POLICY; + private ScheduledExecutorService scheduledExecutor; + + public Builder withRetryPolicy(RetryPolicy retryPolicy) { + this.retryPolicy = retryPolicy; + return this; + } + + public Builder withScheduledExecutor(ScheduledExecutorService scheduledExecutor) { + this.scheduledExecutor = scheduledExecutor; + return this; + } + + public GroupMembershipSettings build() { + return new GroupMembershipSettings(this); + } + } } + diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java new file mode 100644 index 000000000..fcd3b774a --- /dev/null +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java @@ -0,0 +1,103 @@ +package tech.ydb.coordination.recipes.util; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.core.Status; + +public class RetryableTask { + private static final Logger logger = LoggerFactory.getLogger(RetryableTask.class); + + private final String taskName; + private final Supplier> taskSupplier; + private final ScheduledExecutorService executor; + private final RetryPolicy retryPolicy; + private final long startTime; + private int retryCount; + + public RetryableTask( + String taskName, + Supplier> taskSupplier, + ScheduledExecutorService executor, + RetryPolicy retryPolicy + ) { + this.taskName = taskName; + this.taskSupplier = taskSupplier; + this.executor = executor; + this.retryPolicy = retryPolicy; + this.startTime = System.currentTimeMillis(); + this.retryCount = 0; + } + + public CompletableFuture execute() { + CompletableFuture result = new CompletableFuture<>(); + attemptTask(result); + return result; + } + + private void attemptTask(CompletableFuture result) { + try { + taskSupplier.get().whenComplete((status, throwable) -> { + if (throwable != null) { + handleFailure(result, throwable); + } else if (status.isSuccess()) { + logSuccess(); + result.complete(status); + } else { + handleFailure( + result, + new RuntimeException("Operation '" + taskName + "' failed with status: " + status) + ); + } + }); + } catch (Exception e) { + handleFailure(result, e); + } + } + + private void handleFailure(CompletableFuture result, Throwable failure) { + long elapsedTime = System.currentTimeMillis() - startTime; + long delayMs = retryPolicy.nextRetryMs(retryCount, elapsedTime); + + if (delayMs >= 0) { + retryCount++; + logRetry(delayMs, failure); + + if (delayMs == 0) { + executor.execute(() -> attemptTask(result)); + } else { + executor.schedule(() -> attemptTask(result), delayMs, TimeUnit.MILLISECONDS); + } + } else { + logFailure(failure); + result.completeExceptionally(failure); + } + } + + private void logSuccess() { + if (retryCount > 0) { + logger.info("Operation '{}' succeeded after {} retries", taskName, retryCount); + } else { + logger.info("Operation '{}' succeeded on first attempt", taskName); + } + } + + private void logRetry(long delayMs, Throwable failure) { + logger.warn( + "Attempt {} of operation '{}' failed ({}). Retrying in {}ms", + retryCount, taskName, failure.getMessage(), delayMs + ); + } + + private void logFailure(Throwable failure) { + logger.error( + "Operation '{}' failed after {} retries. Last error: {}", + taskName, retryCount, failure.getMessage(), failure + ); + } +} diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java index 661739d7c..c57e00555 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SemaphoreObserver.java @@ -4,7 +4,9 @@ import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; @@ -19,6 +21,10 @@ import tech.ydb.core.Result; import tech.ydb.core.Status; +/** + * Observes changes in a distributed semaphore state and notifies listeners. + * Handles automatic reconnection and retries on failures. + */ public class SemaphoreObserver implements Closeable { private static final Logger logger = LoggerFactory.getLogger(SemaphoreObserver.class); @@ -27,73 +33,150 @@ public class SemaphoreObserver implements Closeable { private final WatchSemaphoreMode watchSemaphoreMode; private final DescribeSemaphoreMode describeSemaphoreMode; private final RetryPolicy retryPolicy; + private final ScheduledExecutorService scheduledExecutor; private final ListenableContainer watchDataListenable = new ListenableContainer<>(); private final ListenableContainer sessionStateListenable = new ListenableContainer<>(); private AtomicReference state = new AtomicReference<>(State.CREATED); private AtomicReference cachedData = new AtomicReference<>(null); private Future watchTask; + private final AtomicReference> forceDescribeTask = new AtomicReference<>(); + /** + * Observer state + */ + public enum State { + CREATED, + STARTED, + CLOSED + } + + /** + * Creates a new semaphore observer instance. + * + * @param session coordination session to use + * @param semaphoreName name of the semaphore to observe + * @param watchSemaphoreMode watch mode configuration + * @param describeSemaphoreMode describe mode configuration + * @param retryPolicy retry policy for failed operations + * @param scheduledExecutor executor for scheduling retries + */ public SemaphoreObserver( CoordinationSession session, String semaphoreName, WatchSemaphoreMode watchSemaphoreMode, DescribeSemaphoreMode describeSemaphoreMode, - RetryPolicy retryPolicy + RetryPolicy retryPolicy, + ScheduledExecutorService scheduledExecutor ) { this.session = Objects.requireNonNull(session, "session cannot be null"); this.semaphoreName = Objects.requireNonNull(semaphoreName, "semaphoreName cannot be null"); - this.watchSemaphoreMode = watchSemaphoreMode; - this.describeSemaphoreMode = describeSemaphoreMode; - this.retryPolicy = retryPolicy; + this.watchSemaphoreMode = Objects.requireNonNull(watchSemaphoreMode, "watchSemaphoreMode cannot be null"); + this.describeSemaphoreMode = Objects.requireNonNull( + describeSemaphoreMode, "describeSemaphoreMode cannot be null" + ); + this.retryPolicy = Objects.requireNonNull(retryPolicy, "retryPolicy cannot be null"); + this.scheduledExecutor = Objects.requireNonNull(scheduledExecutor, "scheduledExecutor cannot be null"); this.session.addStateListener(state -> { - if (!state.isActive()) { + logger.debug("Session state changed to {} for semaphore {}", state, semaphoreName); + if (state == CoordinationSession.State.LOST || state == CoordinationSession.State.CLOSED) { + logger.warn("Session lost or closed, closing observer for semaphore {}", semaphoreName); close(); } - // TODO: clear data after reconnect? + + if (state == CoordinationSession.State.RECONNECTED) { + logger.info("Session reconnected, forcing describe for semaphore {}", semaphoreName); + enqueueForceDescribe(); + } + sessionStateListenable.notifyListeners(state); }); } - public enum State { - CREATED, - STARTED, - CLOSED + /** + * Starts observing the semaphore state. + * Can only be called once. + */ + public void start() { + if (!state.compareAndSet(State.CREATED, State.STARTED)) { + logger.warn("Attempt to start already started observer for semaphore {}", semaphoreName); + return; + } + + logger.info("Starting semaphore observer for: {}", semaphoreName); + enqueueForceDescribe(); + enqueueWatch(); } - public void start() { - if (state.compareAndSet(State.CREATED, State.STARTED)) { - // TODO: first describe - enqueueWatch(); + private void enqueueForceDescribe() { + CompletableFuture existingTask = forceDescribeTask.get(); + if (existingTask != null && !existingTask.isDone()) { + logger.debug("Force describe task already exists for semaphore {}", semaphoreName); + return; + } + + Supplier> taskSupplier = () -> + session.describeSemaphore(semaphoreName, describeSemaphoreMode) + .thenApply(result -> { + if (result.isSuccess()) { + logger.debug("Successfully described semaphore {}", semaphoreName); + saveDescription(result.getValue()); + } else { + logger.warn("Failed to describe semaphore {}: {}", semaphoreName, result.getStatus()); + } + return result.getStatus(); + }); + + CompletableFuture newTask = new RetryableTask( + "semaphoreObserverForceDescribe-" + semaphoreName, + taskSupplier, + scheduledExecutor, + retryPolicy + ).execute(); + + if (!forceDescribeTask.compareAndSet(existingTask, newTask)) { + newTask.cancel(true); + logger.debug("Another thread updated force describe task first for semaphore {}", semaphoreName); } } private synchronized boolean enqueueWatch() { if (watchTask != null && state.get() == State.STARTED) { + logger.warn("Watch task already exists for semaphore {}", semaphoreName); return false; } - watchTask = watchSemaphore().thenCompose(status -> { + logger.debug("Enqueuing new watch task for semaphore {}", semaphoreName); + CompletableFuture watchRetriedTask = new RetryableTask( + "semaphoreObserverWatchTask-" + semaphoreName, + this::watchSemaphore, + scheduledExecutor, + retryPolicy + ).execute(); + + this.watchTask = watchRetriedTask.thenCompose(status -> { if (!status.isSuccess()) { - // TODO: backoff via retryPolicy logger.error("Failed to watch semaphore: {} with status: {}", semaphoreName, status); } - - finish(); + finishWatch(); return null; }); + return true; } - private synchronized void finish() { + private synchronized void finishWatch() { + logger.debug("Finishing watch task for semaphore {}", semaphoreName); watchTask = null; if (state.get() == State.STARTED) { + logger.debug("Restarting watch for semaphore {}", semaphoreName); enqueueWatch(); } } private CompletableFuture watchSemaphore() { + logger.debug("Starting watch operation for semaphore {}", semaphoreName); return session.watchSemaphore( semaphoreName, describeSemaphoreMode, @@ -101,39 +184,71 @@ private CompletableFuture watchSemaphore() { ).thenCompose(result -> { Status status = result.getStatus(); if (!status.isSuccess()) { + logger.warn("Watch operation failed for semaphore {}: {}", semaphoreName, status); return CompletableFuture.completedFuture(status); } + SemaphoreWatcher watcher = result.getValue(); - saveWatchState(watcher.getDescription()); + saveDescription(watcher.getDescription()); + logger.debug("Successfully started watching semaphore {}", semaphoreName); return watcher.getChangedFuture().thenApply(Result::getStatus); }); } - private void saveWatchState(SemaphoreDescription description) { - logger.info("Changed semaphore state from {} to {}", cachedData.get(), description); - cachedData.set(description); + private void saveDescription(SemaphoreDescription description) { + SemaphoreDescription prev = cachedData.getAndSet(description); + logger.info("Semaphore state changed: {} -> {}", + formatSemaphoreDescription(prev), + formatSemaphoreDescription(description)); watchDataListenable.notifyListeners(description); } - private synchronized void stopTaskInternal() { - Future localWatchTask = watchTask; - if (localWatchTask != null) { - localWatchTask.cancel(true); - watchTask = null; + private static String formatSemaphoreDescription(SemaphoreDescription desc) { + if (desc == null) { + return "null"; } + return String.format("SemaphoreDescription{name='%s', count=%d, limit=%d, owners=%d, waiters=%d}", + desc.getName(), desc.getCount(), desc.getLimit(), + desc.getOwnersList().size(), desc.getWaitersList().size()); } + /** + * Returns listenable for semaphore state changes. + */ public Listenable getWatchDataListenable() { return watchDataListenable; } + /** + * Gets the last observed semaphore state. + * @return last cached semaphore description or null if not available + */ public @Nullable SemaphoreDescription getCachedData() { return cachedData.get(); } + /** + * Closes the observer and releases all resources. + */ @Override public void close() { state.set(State.CLOSED); - stopTaskInternal(); + stopTasks(); + } + + private synchronized void stopTasks() { + Future localWatchTask = watchTask; + CompletableFuture describeTask = forceDescribeTask.getAndSet(null); + + if (describeTask != null) { + logger.debug("Cancelling force describe task for semaphore {}", semaphoreName); + describeTask.cancel(true); + } + + if (localWatchTask != null) { + logger.debug("Cancelling watch task for semaphore {}", semaphoreName); + localWatchTask.cancel(true); + watchTask = null; + } } } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java index bea966048..c14997d55 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java @@ -12,7 +12,6 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,7 +105,6 @@ public void takeLeadership() throws Exception { } @Test - @Ignore public void shouldTrackParticipantsAndLeader() throws Exception { String testName = "shouldTrackParticipantsAndLeader"; diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java index e93181ad6..8d8125425 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java @@ -9,7 +9,6 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import tech.ydb.common.retry.RetryForever; import tech.ydb.coordination.CoordinationClient; import tech.ydb.test.junit4.GrpcTransportRule; @@ -45,8 +44,7 @@ private GroupMembershipImpl getGroupMembership( return new GroupMembershipImpl( client, coordinationNodePath, - groupName, - new RetryForever(100) + groupName ); } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java index 8ccc7638b..f0425cda8 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java @@ -1,5 +1,7 @@ package tech.ydb.coordination.recipes.util; +import java.util.concurrent.Executors; + import org.junit.Test; import tech.ydb.common.retry.RetryForever; import tech.ydb.coordination.CoordinationSessionBaseMockedTest; @@ -19,7 +21,8 @@ public void successTest() { "observable_semaphore", WatchSemaphoreMode.WATCH_DATA_AND_OWNERS, DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, - new RetryForever(100) + new RetryForever(100), + Executors.newSingleThreadScheduledExecutor() ); observer.start(); sessionMock.connected(); From e3e429d826300235132dd43054bdab2b857d090f Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Tue, 6 May 2025 00:50:02 +0300 Subject: [PATCH 06/10] added retryable task test && refactored util module --- .../coordination/recipes/util/Listenable.java | 24 ++- .../recipes/util/ListenableContainer.java | 49 ++++++- .../recipes/util/RetryableTask.java | 2 +- .../util/SessionListenableProvider.java | 8 + .../recipes/util/RetryableTaskTest.java | 137 ++++++++++++++++++ 5 files changed, 208 insertions(+), 12 deletions(-) create mode 100644 coordination/src/test/java/tech/ydb/coordination/recipes/util/RetryableTaskTest.java diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java index 6097d0b0b..2cc31bd38 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/Listenable.java @@ -3,15 +3,33 @@ import java.util.concurrent.ExecutorService; import java.util.function.Consumer; +/** + * Generic interface for objects that allow adding and removing listeners for events of type T. + * + * @param the type of event data that listeners will receive + */ public interface Listenable { + /** + * Adds a listener that will be notified synchronously when the event occurs. + * + * @param listener the listener to add, must not be null + * @throws NullPointerException if listener is null + */ void addListener(Consumer listener); /** - * Listener call will be processed in executor - * @param listener - * @param executor + * Adds a listener that will be notified asynchronously using the provided executor. + * + * @param listener the listener to add, must not be null + * @param executor the executor to use for asynchronous notification, must not be null + * @throws NullPointerException if listener or executor is null */ void addListener(Consumer listener, ExecutorService executor); + /** + * Removes the specified listener. + * + * @param listener the listener to remove + */ void removeListener(Consumer listener); } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java index 8e1c788f3..9769ee5b4 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/ListenableContainer.java @@ -1,6 +1,7 @@ package tech.ydb.coordination.recipes.util; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.function.Consumer; @@ -8,24 +9,42 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Thread-safe container for managing and notifying listeners. + * + * @param the type of event data that listeners will receive + */ public class ListenableContainer implements Listenable { private static final Logger logger = LoggerFactory.getLogger(ListenableContainer.class); + // Maps original listeners to potentially wrapped listeners private final Map, Consumer> listenersMapping = new ConcurrentHashMap<>(); + /** + * Notifies all registered listeners with the provided data. + * Exceptions thrown by listeners are caught and logged. + * + * @param data the data to send to listeners + * @throws NullPointerException if data is null + */ public void notifyListeners(T data) { + Objects.requireNonNull(data, "Data cannot be null"); + listenersMapping.values().forEach(listener -> { try { listener.accept(data); } catch (Exception ex) { - logger.error("Listener threw exception", ex); + logger.error("Listener threw exception during notification", ex); } }); } @Override - public void addListener(Consumer listener) { + public void addListener(Consumer listener) { + Objects.requireNonNull(listener, "Listener cannot be null"); + if (listenersMapping.containsKey(listener)) { + logger.debug("Listener already registered, skipping"); return; } @@ -33,22 +52,36 @@ public void addListener(Consumer listener) { } @Override - public void addListener(Consumer listener, ExecutorService executor) { + public void addListener(Consumer listener, ExecutorService executor) { + Objects.requireNonNull(listener, "Listener cannot be null"); + Objects.requireNonNull(executor, "Executor cannot be null"); + if (listenersMapping.containsKey(listener)) { + logger.debug("Listener already registered, skipping"); return; } - Consumer wrapper = new Consumer() { - @Override - public void accept(T data) { - executor.submit(() -> listener.accept(data)); + Consumer wrapper = data -> { + try { + executor.submit(() -> { + try { + listener.accept(data); + } catch (Exception ex) { + logger.error("Asynchronous listener threw exception", ex); + } + }); + } catch (Exception ex) { + logger.error("Failed to submit listener task to executor", ex); } }; + listenersMapping.put(listener, wrapper); } @Override - public void removeListener(Consumer listener) { + public void removeListener(Consumer listener) { + Objects.requireNonNull(listener, "Listener cannot be null"); + listenersMapping.remove(listener); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java index fcd3b774a..1e7ccf43e 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java @@ -40,7 +40,7 @@ public CompletableFuture execute() { return result; } - private void attemptTask(CompletableFuture result) { + void attemptTask(CompletableFuture result) { try { taskSupplier.get().whenComplete((status, throwable) -> { if (throwable != null) { diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java index 5d455ad3e..293996d19 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/SessionListenableProvider.java @@ -2,6 +2,14 @@ import tech.ydb.coordination.CoordinationSession; +/** + * Provides access to a Listenable for session state changes. + */ public interface SessionListenableProvider { + /** + * Gets the Listenable for session state changes. + * + * @return the Listenable instance for session state changes, never null + */ Listenable getSessionListenable(); } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/util/RetryableTaskTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/util/RetryableTaskTest.java new file mode 100644 index 000000000..a225951e9 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/util/RetryableTaskTest.java @@ -0,0 +1,137 @@ +package tech.ydb.coordination.recipes.util; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.core.Status; +import tech.ydb.core.StatusCode; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +@RunWith(MockitoJUnitRunner.class) +public class RetryableTaskTest { + @Mock + private Supplier> taskSupplier; + + @Mock + private ScheduledExecutorService executor; + + @Mock + private RetryPolicy retryPolicy; + + private RetryableTask retryableTask; + private final String taskName = "testTask"; + + @Before + public void setUp() { + retryableTask = new RetryableTask(taskName, taskSupplier, executor, retryPolicy); + } + + @Test + public void testExecute_SuccessOnFirstAttempt() { + Status successStatus = Status.SUCCESS; + CompletableFuture future = CompletableFuture.completedFuture(successStatus); + + when(taskSupplier.get()).thenReturn(future); + + CompletableFuture result = retryableTask.execute(); + + assertTrue(result.isDone()); + assertEquals(successStatus, result.join()); + } + + @Test + public void testExecute_FailureWithRetries() { + Status failureStatus = Status.of(StatusCode.CLIENT_INTERNAL_ERROR); + RuntimeException exception = new RuntimeException("Operation failed"); + + // First attempt fails + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + + when(taskSupplier.get()) + .thenReturn(failedFuture) + .thenReturn(CompletableFuture.completedFuture(failureStatus)); + + when(retryPolicy.nextRetryMs(anyInt(), anyLong())) + .thenReturn(100L) // First retry after 100ms + .thenReturn(-1L); // No more retries + + CompletableFuture result = retryableTask.execute(); + + // Verify retry was scheduled + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + + // Simulate retry execution + retryableTask.attemptTask(result); + + assertTrue(result.isDone()); + assertTrue(result.isCompletedExceptionally()); + } + + @Test + public void testExecute_SuccessAfterRetry() { + Status successStatus = Status.SUCCESS; + RuntimeException exception = new RuntimeException("Temporary failure"); + + // First attempt fails + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + + when(taskSupplier.get()) + .thenReturn(failedFuture) + .thenReturn(CompletableFuture.completedFuture(successStatus)); + + when(retryPolicy.nextRetryMs(anyInt(), anyLong())) + .thenReturn(0L); // Immediate retry + + CompletableFuture result = retryableTask.execute(); + + // Verify immediate retry was scheduled + verify(executor).execute(any(Runnable.class)); + + // Simulate retry execution + retryableTask.attemptTask(result); + + assertTrue(result.isDone()); + assertEquals(successStatus, result.join()); + } + + @Test + public void testExecute_NoMoreRetries() { + RuntimeException exception = new RuntimeException("Permanent failure"); + + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + + when(taskSupplier.get()).thenReturn(failedFuture); + when(retryPolicy.nextRetryMs(anyInt(), anyLong())).thenReturn(-1L); // No more retries + + CompletableFuture result = retryableTask.execute(); + + assertTrue(result.isDone()); + assertTrue(result.isCompletedExceptionally()); + } + + @Test + public void testExecute_TaskSupplierThrowsException() { + RuntimeException exception = new RuntimeException("Supplier failure"); + + when(taskSupplier.get()).thenThrow(exception); + when(retryPolicy.nextRetryMs(anyInt(), anyLong())).thenReturn(-1L); // No more retries + + CompletableFuture result = retryableTask.execute(); + + assertTrue(result.isDone()); + assertTrue(result.isCompletedExceptionally()); + } +} From b43bd6519fefa4bd19a3efbcd8f6463e77b9ba94 Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Tue, 6 May 2025 01:28:20 +0300 Subject: [PATCH 07/10] finished rwlock --- .../recipes/election/LeaderElection.java | 2 +- .../recipes/locks/InterProcessLock.java | 4 +- .../recipes/locks/InterProcessMutex.java | 43 ++--- .../locks/ReadWriteInterProcessLock.java | 176 +++++++++++++++--- ...dWriteInterProcessLockIntegrationTest.java | 5 +- 5 files changed, 183 insertions(+), 47 deletions(-) diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java index 42005917d..ca51223d5 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -277,7 +277,7 @@ private void waitStartedState() throws InterruptedException { localState = state.get(); } - if (localState == State.INITIAL || localState == State.CLOSED || localState == State.FAILED) { + if (localState != State.STARTED) { throw new IllegalStateException("Unexpected state: " + localState.name()); } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java index 813649541..d32508462 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessLock.java @@ -36,11 +36,11 @@ boolean acquire(Duration waitDuration) throws Exception, LockAlreadyAcquiredExce * Releases the lock if it is held by this process. * * @return false if there was nothing to release - * @throws InterruptedException if the thread is interrupted + * @throws Exception if an unexpected error occurs * @throws LockReleaseFailedException if the lock release fails * @throws LockStateException if the lock is in invalid state for release */ - boolean release() throws InterruptedException, LockReleaseFailedException, LockStateException; + boolean release() throws Exception, LockReleaseFailedException, LockStateException; /** diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java index ee79c78b9..6929723fc 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java @@ -3,6 +3,7 @@ import java.io.Closeable; import java.time.Duration; import java.time.Instant; +import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; @@ -24,12 +25,14 @@ public class InterProcessMutex implements InterProcessLock, Closeable { private static final Logger logger = LoggerFactory.getLogger(InterProcessMutex.class); - private final AtomicReference state = new AtomicReference<>(State.INITIAL); + private final String lockName; private final CoordinationSession coordinationSession; private final Future sessionConnectionTask; private final LockInternals lockInternals; private final ListenableContainer sessionListenable; + private final AtomicReference state = new AtomicReference<>(State.INITIAL); + /** * Internal state machine states */ @@ -82,15 +85,14 @@ public InterProcessMutex( } state.set(State.STARTING); - logger.debug("Initializing InterProcessMutex for lock '{}'", lockName); - + this.lockName = lockName; this.coordinationSession = client.createSession(coordinationNodePath); this.sessionListenable = new ListenableContainer<>(); this.lockInternals = new LockInternals(coordinationSession, lockName); coordinationSession.addStateListener(sessionState -> { if (sessionState == CoordinationSession.State.LOST || sessionState == CoordinationSession.State.CLOSED) { - logger.error("Coordination session unexpectedly changed to {} state, marking mutex as FAILED", + logger.error("Coordination session unexpectedly changed to '{}' state, marking mutex as 'FAILED'", sessionState); state.set(State.FAILED); } @@ -109,9 +111,8 @@ public InterProcessMutex( if (settings.isWaitConnection()) { try { - logger.debug("Waiting for session connection to complete..."); + logger.debug("Waiting for session connection to complete for lock '{}'", lockName); sessionConnectionTask.get(); - logger.debug("Session connection completed"); } catch (InterruptedException e) { Thread.currentThread().interrupt(); logger.error("Interrupted while waiting for session connection for lock '{}'", lockName, e); @@ -128,36 +129,38 @@ public InterProcessMutex( @Override public void acquire() throws Exception { checkState(); - logger.debug("Attempting to acquire lock..."); + logger.debug("Attempting to acquire lock '{}'", lockName); lockInternals.tryAcquire( null, true, null ); - logger.debug("Lock acquired successfully"); + logger.debug("Lock '{}' acquired successfully", lockName); } @Override public boolean acquire(Duration waitDuration) throws Exception { + Objects.requireNonNull(waitDuration, "wait duration must not be null"); + checkState(); - logger.debug("Attempting to acquire lock with timeout {}...", waitDuration); + logger.debug("Attempting to acquire lock '{}' with timeout {}", lockName, waitDuration); Instant deadline = Instant.now().plus(waitDuration); boolean acquired = lockInternals.tryAcquire( deadline, true, null ) != null; - logger.debug("Lock acquisition {}successful", acquired ? "" : "un"); + logger.debug("Lock '{}' acquisition {}successful", lockName, acquired ? "" : "un"); return acquired; } @Override public boolean release() throws InterruptedException { checkState(); - logger.debug("Attempting to release lock..."); + logger.debug("Attempting to release lock '{}'", lockName); boolean released = lockInternals.release(); if (released) { - logger.debug("Lock released successfully"); + logger.debug("Lock '{}' released successfully", lockName); } else { logger.debug("No lock to release"); } @@ -184,29 +187,27 @@ public Listenable getSessionListenable() { @Override public void close() { - logger.debug("Closing InterProcessMutex..."); + logger.debug("Closing mutex '{}'", lockName); state.set(State.CLOSED); try { lockInternals.close(); } catch (Exception e) { - logger.warn("Error while closing lock internals", e); + logger.warn("Error while closing lock internals '{}'", lockName, e); } - logger.info("InterProcessMutex closed"); + logger.info("Mutex '{}' closed", lockName); } private void checkState() throws LockStateException { State currentState = state.get(); if (currentState == State.FAILED) { - throw new LockStateException("Lock '" + lockInternals.getLockName() + "' is in FAILED state", - lockInternals.getLockName()); + throw new LockStateException("Lock '" + lockName + "' is in FAILED state", lockName); } if (currentState == State.CLOSED) { - throw new LockStateException("Lock '" + lockInternals.getLockName() + "' is already closed", - lockInternals.getLockName()); + throw new LockStateException("Lock '" + lockName + "' is already closed", lockName); } if (currentState != State.STARTED) { - throw new LockStateException("Lock '" + lockInternals.getLockName() + "' is not ready (current state: " - + currentState + ")", lockInternals.getLockName()); + throw new LockStateException("Lock '" + lockName + "' is not ready (current state: " + currentState + ")", + lockName); } } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java index 76649288d..61897d404 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java @@ -4,6 +4,8 @@ import java.time.Duration; import java.time.Instant; import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; import org.slf4j.Logger; @@ -11,16 +13,34 @@ import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.recipes.locks.exception.LockInitializationException; +import tech.ydb.coordination.recipes.locks.exception.LockStateException; import tech.ydb.coordination.recipes.util.Listenable; import tech.ydb.coordination.recipes.util.ListenableContainer; -// TODO: state management + (документация, логи, рекомендации) +/** + * Distributed read-write lock implementation that allows multiple readers or a single writer. + * + *

This implementation provides non-reentrant read/write locking semantics across multiple processes. + * Multiple processes can hold the read lock simultaneously, while only one process can hold + * the write lock (with no concurrent readers).

+ * + *

Thread-safety: Instances of this class are thread-safe and can be used from multiple threads.

+ */ public class ReadWriteInterProcessLock implements Closeable { private static final Logger logger = LoggerFactory.getLogger(ReadWriteInterProcessLock.class); private final InternalLock readLock; private final InternalLock writeLock; + /** + * Creates a new distributed read-write lock with default settings. + * + * @param client the coordination client instance + * @param coordinationNodePath the base path for coordination nodes + * @param lockName the name of the lock (must be unique within the coordination namespace) + * @throws IllegalArgumentException if any parameter is null + */ public ReadWriteInterProcessLock( CoordinationClient client, String coordinationNodePath, @@ -34,32 +54,93 @@ public ReadWriteInterProcessLock( ); } + /** + * Creates a new distributed read-write lock with custom settings. + * + * @param client the coordination client instance + * @param coordinationNodePath the base path for coordination nodes + * @param lockName the name of the lock (must be unique within the coordination namespace) + * @param settings the lock configuration settings + * @throws IllegalArgumentException if any parameter is null + * @throws LockInitializationException if the lock cannot be initialized + */ public ReadWriteInterProcessLock( CoordinationClient client, String coordinationNodePath, String lockName, ReadWriteInterProcessLockSettings settings ) { - this.readLock = new InternalLock(client, coordinationNodePath, lockName, false); - this.writeLock = new InternalLock(client, coordinationNodePath, lockName, true); + if (client == null || coordinationNodePath == null || lockName == null || settings == null) { + throw new IllegalArgumentException("All parameters must be non-null"); + } + + this.readLock = new InternalLock( + client, + coordinationNodePath, + lockName, + false + ); + this.writeLock = new InternalLock( + client, + coordinationNodePath, + lockName, + true + ); + + if (settings.isWaitConnection()) { + try { + logger.debug("Waiting for session connection to complete for rwlock {}", lockName); + readLock.sessionConnectionTask.get(); + writeLock.sessionConnectionTask.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while waiting for session connection for rwlock '{}'", lockName, e); + throw new LockInitializationException( + "Interrupted while initializing rwlock '" + lockName + "'", e, lockName + ); + } catch (ExecutionException e) { + logger.error("Failed to initialize rwlock '{}' due to execution error", lockName, e); + throw new LockInitializationException( + "Failed to initialize rwlock '" + lockName + "'", e.getCause(), lockName + ); + } + } } + /** + * Returns the write lock instance. + * + * @return the write lock (exclusive) + */ public InterProcessLock writeLock() { return writeLock; } + /** + * Returns the read lock instance. + * + * @return the read lock (shared) + */ public InterProcessLock readLock() { return readLock; } + /** + * Internal lock implementation that handles both read and write operations. + */ private static class InternalLock implements InterProcessLock { + private final String lockName; private final boolean isExclusive; - - private final AtomicReference state = new AtomicReference<>(State.INITIAL); + private final Future sessionConnectionTask; private final CoordinationSession coordinationSession; private final LockInternals lockInternals; private final ListenableContainer sessionListenable; + private final AtomicReference state = new AtomicReference<>(State.INITIAL); + + /** + * Internal state of the lock. + */ private enum State { INITIAL, STARTING, @@ -74,49 +155,75 @@ public InternalLock( String lockName, boolean isExclusive ) { - this.isExclusive = isExclusive; + state.set(State.STARTING); + logger.debug("Initializing InterProcessMutex for lock '{}'", lockName); + this.lockName = lockName; this.coordinationSession = client.createSession(coordinationNodePath); - this.lockInternals = new LockInternals( - coordinationSession, - lockName - ); this.sessionListenable = new ListenableContainer<>(); - coordinationSession.addStateListener(sessionListenable::notifyListeners); - // TODO: add settings to block - coordinationSession.connect().thenAccept(status -> { - status.expectSuccess("Unable to establish session"); + this.lockInternals = new LockInternals(coordinationSession, lockName); + this.isExclusive = isExclusive; + + coordinationSession.addStateListener(sessionState -> { + if (sessionState == CoordinationSession.State.LOST || + sessionState == CoordinationSession.State.CLOSED) { + logger.error("Coordination session unexpectedly changed to {} state, marking lock as FAILED", + sessionState); + state.set(State.FAILED); + } + sessionListenable.notifyListeners(sessionState); }); - } - private void start() { - ; + sessionConnectionTask = coordinationSession.connect().thenAccept(sessionConnectStatus -> { + if (!sessionConnectStatus.isSuccess()) { + state.set(State.FAILED); + logger.error("Failed to establish coordination session for lock '{}'", lockName); + } else { + state.set(State.STARTED); + logger.info("Successfully established session for lock '{}'", lockName); + } + }); } @Override public void acquire() throws Exception { + checkState(); + logger.debug("Attempting to acquire lock {}", lockName); lockInternals.tryAcquire( null, isExclusive, null ); + logger.debug("Lock '{}' acquired successfully", lockName); } @Override public boolean acquire(Duration waitDuration) throws Exception { Objects.requireNonNull(waitDuration, "wait duration must not be null"); + checkState(); + logger.debug("Attempting to acquire lock '{}' with timeout {}", lockName, waitDuration); Instant deadline = Instant.now().plus(waitDuration); - return lockInternals.tryAcquire( + boolean acquired = lockInternals.tryAcquire( deadline, isExclusive, null ) != null; + logger.debug("Lock '{}' acquisition {}successful", lockName, acquired ? "" : "un"); + return acquired; } @Override - public boolean release() throws InterruptedException { - return lockInternals.release(); + public boolean release() throws Exception { + checkState(); + logger.debug("Attempting to release lock '{}'", lockName); + boolean released = lockInternals.release(); + if (released) { + logger.debug("Lock {} released successfully", lockName); + } else { + logger.debug("No lock to release"); + } + return released; } @Override @@ -130,14 +237,39 @@ public Listenable getSessionListenable() { } private void close() { - lockInternals.close(); + logger.debug("Closing rwlock {}", lockName); + state.set(State.CLOSED); + try { + lockInternals.close(); + } catch (Exception e) { + logger.warn("Error while closing rwlock internals {}", lockName, e); + } + logger.info("Rwlock {} closed", lockName); + } + + private void checkState() throws LockStateException { + State currentState = state.get(); + if (currentState == State.FAILED) { + throw new LockStateException("Lock '" + lockName + "' is in FAILED state", lockName); + } + if (currentState == State.CLOSED) { + throw new LockStateException("Lock '" + lockName + "' is already closed", lockName); + } + if (currentState != State.STARTED) { + throw new LockStateException( + "Lock '" + lockName + "' is not ready (current state: " + currentState + ")", lockName + ); + } } } + /** + * Closes both read and write locks and releases all associated resources. + * After closing, the lock instance can no longer be used. + */ @Override public void close() { readLock.close(); writeLock.close(); } - } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java index d7e17a9c3..6459eeb57 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java @@ -295,7 +295,10 @@ private ReadWriteInterProcessLock getReadWriteLock(String nodePath, String lockN return new ReadWriteInterProcessLock( client, nodePath, - lockName + lockName, + ReadWriteInterProcessLockSettings.newBuilder() + .withWaitConnection(true) + .build() ); } From 09fa1cde4f1bf1401f48842ac7aecae7cf66ae34 Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Tue, 6 May 2025 17:15:30 +0300 Subject: [PATCH 08/10] added documentation to leader election --- .../recipes/election/LeaderElection.java | 153 +++++++++++++++--- .../election/LeaderElectionListener.java | 61 +++++++ .../recipes/group/GroupMembership.java | 148 ++++++++++++++--- .../recipes/group/GroupMembershipImpl.java | 139 ---------------- .../group/GroupMembershipIntegrationTest.java | 8 +- 5 files changed, 330 insertions(+), 179 deletions(-) delete mode 100644 coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java index ca51223d5..67a9d9820 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -11,12 +11,14 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,9 +37,27 @@ import tech.ydb.core.Status; import tech.ydb.core.StatusCode; -// TODO: документцаия / логгирование / рекомендации по коду +/** + * A distributed leader election implementation using coordination services. + * This class provides a mechanism for multiple instances to compete for leadership + * of a named resource, with exactly one instance becoming the leader at any time. + * + *

The election process uses a semaphore-based approach where: + *

    + *
  • The leader holds the semaphore lock
  • + *
  • Other participants wait in a queue
  • + *
  • Leadership can be voluntarily released or lost due to session issues
  • + *
+ * + *

Thread safety: This class is thread-safe. All public methods can be called + * from multiple threads concurrently. + */ public class LeaderElection implements Closeable, SessionListenableProvider { private static final Logger logger = LoggerFactory.getLogger(LeaderElection.class); + private static final ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setNameFormat("ydb-leader-election-%d") + .setDaemon(true) + .build(); private static final long MAX_LEASE = 1L; private final LeaderElectionListener leaderElectionListener; @@ -68,6 +88,15 @@ private enum State { CLOSED } + /** + * Creates a new LeaderElection instance with default settings. + * + * @param client the coordination client to use + * @param coordinationNodePath path to the coordination node + * @param electionName name of the election (must be unique per coordination node) + * @param data optional data to associate with the leader (visible to all participants) + * @param leaderElectionListener callback for leadership events + */ public LeaderElection( CoordinationClient client, String coordinationNodePath, @@ -86,6 +115,17 @@ public LeaderElection( ); } + /** + * Creates a new LeaderElection instance with custom settings. + * + * @param client the coordination client to use + * @param coordinationNodePath path to the coordination node + * @param electionName name of the election (must be unique per coordination node) + * @param data optional data to associate with the leader (visible to all participants) + * @param leaderElectionListener callback for leadership events + * @param settings configuration settings for the election process + * @throws NullPointerException if any required parameter is null + */ public LeaderElection( CoordinationClient client, String coordinationNodePath, @@ -94,21 +134,28 @@ public LeaderElection( LeaderElectionListener leaderElectionListener, LeaderElectionSettings settings ) { + Preconditions.checkNotNull(client, "CoordinationClient cannot be null"); + Preconditions.checkNotNull(coordinationNodePath, "Coordination node path cannot be null"); + Preconditions.checkNotNull(electionName, "Election name cannot be null"); + Preconditions.checkNotNull(leaderElectionListener, "LeaderElectionListener cannot be null"); + Preconditions.checkNotNull(settings, "LeaderElectionSettings cannot be null"); + this.coordinationNodePath = coordinationNodePath; this.electionName = electionName; this.data = data; this.leaderElectionListener = leaderElectionListener; this.scheduledExecutor = settings.getScheduledExecutor(); - this.blockingExecutor = Executors.newSingleThreadExecutor(); // TODO: thread factory + this.blockingExecutor = Executors.newSingleThreadExecutor(threadFactory); this.retryPolicy = settings.getRetryPolicy(); this.coordinationSession = client.createSession(coordinationNodePath); this.sessionListenable = new ListenableContainer<>(); coordinationSession.addStateListener(sessionState -> { - if (sessionState == CoordinationSession.State.LOST || sessionState == CoordinationSession.State.CLOSED) { + if (!state.get().equals(State.CLOSED) && (sessionState == CoordinationSession.State.LOST || + sessionState == CoordinationSession.State.CLOSED)) { logger.error("Coordination session unexpectedly changed to {} state, marking election as FAILED", sessionState); - state.set(State.FAILED); + stopInternal(State.FAILED); } sessionListenable.notifyListeners(sessionState); }); @@ -127,6 +174,11 @@ public LeaderElection( ); } + /** + * Starts the leader election process. + * + * @throws IllegalStateException if the election is already started or closed + */ public void start() { Preconditions.checkState( state.compareAndSet(State.INITIAL, State.STARTING), @@ -159,9 +211,7 @@ public void start() { return semaphoreStatus; }).exceptionally(ex -> { logger.error("Leader election initializing task failed", ex); - state.set(State.FAILED); - semaphoreObserver.close(); - startingLatch.countDown(); + stopInternal(State.FAILED); return Status.of(StatusCode.CLIENT_INTERNAL_ERROR); }); @@ -176,10 +226,19 @@ private CompletableFuture executeWithRetry(Supplier localTask = electionTask; if (localTask != null) { @@ -231,11 +296,16 @@ public Void call() throws Exception { return false; } + /** + * Main work loop for leadership acquisition and maintenance. + * + * @throws Exception if the leadership attempt fails + */ private void doWork() throws Exception { isLeader = false; try { - waitStartedState(); + waitStartedStateOrFail(); lock.tryAcquire( null, true, @@ -248,7 +318,7 @@ private void doWork() throws Exception { Thread.currentThread().interrupt(); throw e; } catch (Throwable e) { - logger.debug("takeLeadership exception", e); + logger.error("Unexpected error in takeLeadership", e); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -270,7 +340,7 @@ private void doWork() throws Exception { } } - private void waitStartedState() throws InterruptedException { + private void waitStartedStateOrFail() throws InterruptedException { State localState = state.get(); if (localState == State.STARTING) { startingLatch.await(); @@ -295,9 +365,10 @@ private boolean isQueued() { } /** - * Не гарантированы все, кроме лидера + * Gets all participants in the election. + * Note: Due to observer limitations, waiters may be visible only eventually (after lease changes). * - * @return + * @return list of election participants (owners and visible waiters) */ public List getParticipants() { SemaphoreDescription semaphoreDescription = semaphoreObserver.getCachedData(); @@ -313,6 +384,11 @@ public List getParticipants() { ).collect(Collectors.toList()); } + /** + * Gets the current leader if one exists. + * + * @return Optional containing the current leader, or empty if no leader exists + */ public Optional getCurrentLeader() { SemaphoreDescription semaphoreDescription = semaphoreObserver.getCachedData(); if (semaphoreDescription == null) { @@ -336,18 +412,59 @@ public Listenable getSessionListenable() { return sessionListenable; } + /** + * Closes the leader election and releases all resources. + * After closing, the instance cannot be reused. + */ @Override public synchronized void close() { - // TODO: Учесть все стейты - Preconditions.checkState(state.compareAndSet(State.STARTED, State.CLOSED), "Already closed"); + stopInternal(State.CLOSED); + } + + /** + * Internal method to stop the election with the specified termination state. + * + * @param terminationState the state to transition to (FAILED or CLOSED) + * @return true if the state was changed, false if already terminated + */ + private synchronized boolean stopInternal(State terminationState) { + State localState = state.get(); + if (localState == State.FAILED || localState == State.CLOSED) { + logger.warn("Already stopped leader election {} with status: {}", electionName, localState); + return false; + } + logger.debug("Transitioning leader election {} from {} to {}", electionName, localState, terminationState); + + // change state + state.set(terminationState); + // unblock starting latch if not yet + startingLatch.countDown(); + + // stop tasks + Future localInitializingTask = initializingTask.get(); + if (localInitializingTask != null) { + localInitializingTask.cancel(true); + initializingTask.set(null); + } Future localTask = electionTask; if (localTask != null) { localTask.cancel(true); electionTask = null; } - blockingExecutor.shutdown(); - semaphoreObserver.close(); + // Clean up resources + try { + semaphoreObserver.close(); + } catch (Exception e) { + logger.warn("Error closing semaphore observer for {}: {}", electionName, e.getMessage()); + } + + try { + blockingExecutor.shutdown(); + } catch (Exception e) { + logger.warn("Error shutting down executor for {}: {}", electionName, e.getMessage()); + } + return true; } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java index 30a102477..20210c2b7 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElectionListener.java @@ -1,5 +1,66 @@ package tech.ydb.coordination.recipes.election; +/** + * A listener interface for receiving leadership election events in a distributed system. + * + *

Implementations of this interface are notified when the current process becomes + * the leader in a leader election scenario.

+ * + *

Leadership Lifecycle:

+ *
    + *
  1. Election: The distributed system selects a leader
  2. + *
  3. Takeover: {@code takeLeadership()} is invoked on the elected leader
  4. + *
  5. Execution: The leader performs its duties while maintaining leadership
  6. + *
  7. Termination: When {@code takeLeadership()} completes (either normally or exceptionally), + * the leadership is automatically relinquished and new elections begin
  8. + *
+ * + *

Usage Example:

+ *
{@code
+ * LeaderElectionListener listener = new LeaderElectionListener() {
+ *     public void takeLeadership() throws Exception {
+ *         startServices();
+ *
+ *         // Main leadership work
+ *         while (shouldContinueLeadership()) {
+ *             performLeaderDuties();
+ *         }
+ *
+ *         // Cleanup will trigger automatically when method exits
+ *     }
+ * };
+ * }
+ * + *

Important Implementation Notes:

+ *
    + *
  • The leadership is maintained only while {@code takeLeadership()} is executing
  • + *
  • When the method completes (either normally or by throwing an exception), the leadership + * is automatically released and new elections begin immediately
  • + *
  • For long-running leadership, the method should not return until leadership should end
  • + *
  • To voluntarily relinquish leadership before completing, throw an exception
  • + *
+ * + *

Error Handling: If the implementation throws an exception, the leadership will be + * released and new elections will be triggered, just as with normal completion.

+ */ public interface LeaderElectionListener { + /** + * Called when leadership is acquired by the current process. + * + *

The leadership period lasts exactly as long as this method's execution. When the method + * returns (either normally or exceptionally), the leadership is automatically relinquished + * and new elections begin immediately. + * + *

For continuous leadership, implementations should: + *

    + *
  • Perform all initialization at start
  • + *
  • Enter the main leadership loop
  • + *
  • Only return when leadership should end
  • + *
+ * + * @throws Exception if leadership cannot be maintained or should be terminated early. + * The leadership will be released and new elections will begin when any + * exception is thrown. + */ void takeLeadership() throws Exception; } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java index 7fc4e9ccc..45c49c84c 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java @@ -1,26 +1,138 @@ package tech.ydb.coordination.recipes.group; import java.io.Closeable; +import java.io.IOException; +import java.time.Duration; import java.util.List; +import java.util.concurrent.Future; +import java.util.stream.Collectors; import org.checkerframework.checker.nullness.qual.Nullable; +import tech.ydb.common.retry.RetryPolicy; +import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; +import tech.ydb.coordination.description.SemaphoreDescription; import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.SessionListenableProvider; - -public interface GroupMembership extends Closeable, SessionListenableProvider { - /** - * Adds instance to the group and start observing members - */ - void start(); - - /** - * Get cached members of the group or null - */ - @Nullable - List getCurrentMembers(); - - /** - * Get listenable to subscribe to members list update - */ - Listenable> getMembersListenable(); +import tech.ydb.coordination.recipes.util.ListenableContainer; +import tech.ydb.coordination.recipes.util.SemaphoreObserver; +import tech.ydb.coordination.settings.CoordinationSessionSettings; +import tech.ydb.coordination.settings.DescribeSemaphoreMode; +import tech.ydb.coordination.settings.WatchSemaphoreMode; + +public class GroupMembership implements Closeable { + private static final long MAX_GROUP_SIZE = Long.MAX_VALUE; + + private final CoordinationClient coordinationClient; + private final String coordinationNodePath; + private final String groupName; + private final RetryPolicy retryPolicy; + + private final CoordinationSession session; + private final SemaphoreObserver semaphoreObserver; + private final ListenableContainer sessionStateListenable; + private final ListenableContainer> groupMembersListenable; + + private Future acquireTask; + + public GroupMembership( + CoordinationClient coordinationClient, + String coordinationNodePath, + String groupName + ) { + this( + coordinationClient, + coordinationNodePath, + groupName, + GroupMembershipSettings.newBuilder() + .build() + ); + } + + public GroupMembership( + CoordinationClient coordinationClient, + String coordinationNodePath, + String groupName, + GroupMembershipSettings settings + ) { + this.coordinationClient = coordinationClient; + this.coordinationNodePath = coordinationNodePath; + this.groupName = groupName; + this.retryPolicy = settings.getRetryPolicy(); + + this.session = coordinationClient.createSession( + coordinationNodePath, + CoordinationSessionSettings.newBuilder() + .withRetryPolicy(retryPolicy) + .build() + ); + this.sessionStateListenable = new ListenableContainer<>(); + session.addStateListener(sessionStateListenable::notifyListeners); + + this.semaphoreObserver = new SemaphoreObserver( + session, + groupName, + WatchSemaphoreMode.WATCH_OWNERS, + DescribeSemaphoreMode.WITH_OWNERS, + settings.getRetryPolicy(), + settings.getScheduledExecutor() + ); + this.groupMembersListenable = new ListenableContainer<>(); + semaphoreObserver.getWatchDataListenable().addListener(description -> { + List groupMembers = mapSemaphoreDescriptionToMembersList(description); + groupMembersListenable.notifyListeners(groupMembers); + }); + } + + public void start() { + // TODO: correctly handle failed connection, failed semaphore and failed starts + session.connect().thenAccept(sessionStatus -> { + sessionStatus.expectSuccess("Unable to establish session"); + session.createSemaphore(groupName, MAX_GROUP_SIZE).thenAccept(semaphoreStatus -> { + semaphoreObserver.start(); + enqueueAcquire(); + }); + }); + } + + private void enqueueAcquire() { + // TODO: enqueue acquire task and retry it with retry policy + acquireTask = session.acquireSemaphore(groupName, 1, Duration.ofSeconds(30)); + } + + public @Nullable List getCurrentMembers() { + SemaphoreDescription cachedDescription = semaphoreObserver.getCachedData(); + return mapSemaphoreDescriptionToMembersList(cachedDescription); + } + + private static @Nullable List mapSemaphoreDescriptionToMembersList(SemaphoreDescription description) { + if (description == null) { + return null; + } + + List ownersList = description.getOwnersList(); + return ownersList.stream().map(GroupMembership::mapSessionToGroupMember).collect(Collectors.toList()); + } + + private static GroupMember mapSessionToGroupMember(SemaphoreDescription.Session session) { + return new GroupMember( + session.getId(), + session.getData() + ); + } + + public Listenable getSessionListenable() { + return sessionStateListenable; + } + + public Listenable> getMembersListenable() { + return groupMembersListenable; + } + + @Override + public void close() throws IOException { + session.close(); + semaphoreObserver.close(); + + // TODO: stop acquire task + } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java deleted file mode 100644 index 342d25b85..000000000 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembershipImpl.java +++ /dev/null @@ -1,139 +0,0 @@ -package tech.ydb.coordination.recipes.group; - -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; - -import org.checkerframework.checker.nullness.qual.Nullable; -import tech.ydb.common.retry.RetryPolicy; -import tech.ydb.coordination.CoordinationClient; -import tech.ydb.coordination.CoordinationSession; -import tech.ydb.coordination.description.SemaphoreDescription; -import tech.ydb.coordination.recipes.locks.LockInternals; -import tech.ydb.coordination.recipes.util.Listenable; -import tech.ydb.coordination.recipes.util.ListenableContainer; -import tech.ydb.coordination.recipes.util.SemaphoreObserver; -import tech.ydb.coordination.settings.CoordinationSessionSettings; -import tech.ydb.coordination.settings.DescribeSemaphoreMode; -import tech.ydb.coordination.settings.WatchSemaphoreMode; - -public class GroupMembershipImpl implements GroupMembership { - private static final long MAX_GROUP_SIZE = Long.MAX_VALUE; - - private final CoordinationClient coordinationClient; - private final String coordinationNodePath; - private final String groupName; - private final RetryPolicy retryPolicy; - - private final CoordinationSession session; - private final LockInternals lockInternals; - private final SemaphoreObserver semaphoreObserver; - private final ListenableContainer sessionStateListenable; - private final ListenableContainer> groupMembersListenable; - - public GroupMembershipImpl( - CoordinationClient coordinationClient, - String coordinationNodePath, - String groupName - ) { - this( - coordinationClient, - coordinationNodePath, - groupName, - GroupMembershipSettings.newBuilder() - .build() - ); - } - - public GroupMembershipImpl( - CoordinationClient coordinationClient, - String coordinationNodePath, - String groupName, - GroupMembershipSettings settings - ) { - this.coordinationClient = coordinationClient; - this.coordinationNodePath = coordinationNodePath; - this.groupName = groupName; - this.retryPolicy = settings.getRetryPolicy(); - - this.session = coordinationClient.createSession( - coordinationNodePath, - CoordinationSessionSettings.newBuilder() - .withRetryPolicy(retryPolicy) - .build() - ); - this.sessionStateListenable = new ListenableContainer<>(); - session.addStateListener(sessionStateListenable::notifyListeners); - - this.lockInternals = new LockInternals( - session, - groupName, - MAX_GROUP_SIZE - ); - - this.semaphoreObserver = new SemaphoreObserver( - session, - groupName, - WatchSemaphoreMode.WATCH_OWNERS, - DescribeSemaphoreMode.WITH_OWNERS, - settings.getRetryPolicy(), - settings.getScheduledExecutor() - ); - this.groupMembersListenable = new ListenableContainer<>(); - semaphoreObserver.getWatchDataListenable().addListener(description -> { - List groupMembers = mapSemaphoreDescriptionToMembersList(description); - groupMembersListenable.notifyListeners(groupMembers); - }); - } - - @Override - public void start() { - // TODO: correctly handle failed connection, failed semaphore and failed starts - session.connect().thenAccept(sessionStatus -> { - sessionStatus.expectSuccess("Unable to establish session"); - session.createSemaphore(groupName, MAX_GROUP_SIZE).thenAccept(semaphoreStatus -> { - // TODO: start acquiring task - semaphoreObserver.start(); - }); - }); - } - - @Override - public @Nullable List getCurrentMembers() { - SemaphoreDescription cachedDescription = semaphoreObserver.getCachedData(); - return mapSemaphoreDescriptionToMembersList(cachedDescription); - } - - private static @Nullable List mapSemaphoreDescriptionToMembersList(SemaphoreDescription description) { - if (description == null) { - return null; - } - - List ownersList = description.getOwnersList(); - return ownersList.stream().map(GroupMembershipImpl::mapSessionToGroupMember).collect(Collectors.toList()); - } - - private static GroupMember mapSessionToGroupMember(SemaphoreDescription.Session session) { - return new GroupMember( - session.getId(), - session.getData() - ); - } - - @Override - public Listenable getSessionListenable() { - return sessionStateListenable; - } - - @Override - public Listenable> getMembersListenable() { - return groupMembersListenable; - } - - @Override - public void close() throws IOException { - session.close(); - lockInternals.close(); - semaphoreObserver.close(); - } -} diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java index 8d8125425..ea5501024 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java @@ -30,18 +30,18 @@ public static void clean() { ydbRule.close(); } - private GroupMembershipImpl getGroupMembership(String testName) { + private GroupMembership getGroupMembership(String testName) { return getGroupMembership(testName, testName); } - private GroupMembershipImpl getGroupMembership( + private GroupMembership getGroupMembership( String coordinationNodePath, String groupName ) { client.createNode(coordinationNodePath).join().expectSuccess( "cannot create coordination node on path: " + coordinationNodePath ); - return new GroupMembershipImpl( + return new GroupMembership( client, coordinationNodePath, groupName @@ -52,7 +52,7 @@ private GroupMembershipImpl getGroupMembership( public void successTest() throws Exception { String testName = "successTest"; - GroupMembershipImpl groupMembership = getGroupMembership(testName); + GroupMembership groupMembership = getGroupMembership(testName); groupMembership.start(); List currentMembers = groupMembership.getCurrentMembers(); From c13bbca76e802d1fd03bb88c3642605bc87fde3e Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Tue, 6 May 2025 19:02:41 +0300 Subject: [PATCH 09/10] finished group membership --- .../recipes/election/LeaderElection.java | 33 +- .../recipes/group/GroupMembership.java | 318 ++++++++++++++++-- .../tech/ydb/coordination/AwaitAssert.java | 46 +++ .../LeaderElectionIntegrationTest.java | 35 +- .../group/GroupMembershipIntegrationTest.java | 13 +- 5 files changed, 377 insertions(+), 68 deletions(-) create mode 100644 coordination/src/test/java/tech/ydb/coordination/AwaitAssert.java diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java index 67a9d9820..afec97eb2 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -74,12 +74,15 @@ public class LeaderElection implements Closeable, SessionListenableProvider { private final SemaphoreObserver semaphoreObserver; private final CountDownLatch startingLatch = new CountDownLatch(1); - private AtomicReference state = new AtomicReference<>(State.INITIAL); - private AtomicReference> initializingTask = new AtomicReference<>(null); + private final AtomicReference state = new AtomicReference<>(State.INITIAL); + private final AtomicReference> initializingTask = new AtomicReference<>(null); private Future electionTask = null; private volatile boolean autoRequeue = false; private volatile boolean isLeader = false; + /** + * Internal state + */ private enum State { INITIAL, STARTING, @@ -91,10 +94,10 @@ private enum State { /** * Creates a new LeaderElection instance with default settings. * - * @param client the coordination client to use - * @param coordinationNodePath path to the coordination node - * @param electionName name of the election (must be unique per coordination node) - * @param data optional data to associate with the leader (visible to all participants) + * @param client the coordination client to use + * @param coordinationNodePath path to the coordination node + * @param electionName name of the election (must be unique per coordination node) + * @param data optional data to associate with the leader (visible to all participants) * @param leaderElectionListener callback for leadership events */ public LeaderElection( @@ -118,12 +121,12 @@ public LeaderElection( /** * Creates a new LeaderElection instance with custom settings. * - * @param client the coordination client to use - * @param coordinationNodePath path to the coordination node - * @param electionName name of the election (must be unique per coordination node) - * @param data optional data to associate with the leader (visible to all participants) + * @param client the coordination client to use + * @param coordinationNodePath path to the coordination node + * @param electionName name of the election (must be unique per coordination node) + * @param data optional data to associate with the leader (visible to all participants) * @param leaderElectionListener callback for leadership events - * @param settings configuration settings for the election process + * @param settings configuration settings for the election process * @throws NullPointerException if any required parameter is null */ public LeaderElection( @@ -185,6 +188,8 @@ public void start() { "Leader election may be started only once" ); + logger.debug("Starting leader election '{}' initialization", electionName); + CompletableFuture connectionTask = executeWithRetry(coordinationSession::connect); CompletableFuture semaphoreCreateTask = executeWithRetry( () -> coordinationSession.createSemaphore(electionName, MAX_LEASE) @@ -459,12 +464,16 @@ private synchronized boolean stopInternal(State terminationState) { } catch (Exception e) { logger.warn("Error closing semaphore observer for {}: {}", electionName, e.getMessage()); } - try { blockingExecutor.shutdown(); } catch (Exception e) { logger.warn("Error shutting down executor for {}: {}", electionName, e.getMessage()); } + try { + coordinationSession.close(); + } catch (Exception e) { + logger.warn("Error closing session for {}: {}", electionName, e.getMessage()); + } return true; } } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java index 45c49c84c..b0368d39b 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java @@ -4,36 +4,88 @@ import java.io.IOException; import java.time.Duration; import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import java.util.stream.Collectors; +import com.google.common.base.Preconditions; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import tech.ydb.common.retry.RetryPolicy; import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.description.SemaphoreDescription; import tech.ydb.coordination.recipes.util.Listenable; import tech.ydb.coordination.recipes.util.ListenableContainer; +import tech.ydb.coordination.recipes.util.RetryableTask; import tech.ydb.coordination.recipes.util.SemaphoreObserver; import tech.ydb.coordination.settings.CoordinationSessionSettings; import tech.ydb.coordination.settings.DescribeSemaphoreMode; import tech.ydb.coordination.settings.WatchSemaphoreMode; +import tech.ydb.core.Result; +import tech.ydb.core.Status; +import tech.ydb.core.StatusCode; +/** + * A distributed group membership implementation that uses coordination service + * to manage membership and track group members. + * + *

This class provides mechanisms to: + *

    + *
  • Join a named group
  • + *
  • Track current group members
  • + *
  • Receive notifications about membership changes
  • + *
+ * + *

The implementation uses a semaphore with watch capabilities to track membership. + */ public class GroupMembership implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(GroupMembership.class); private static final long MAX_GROUP_SIZE = Long.MAX_VALUE; + private static final Duration ACQUIRE_TIMEOUT = Duration.ofSeconds(30); - private final CoordinationClient coordinationClient; - private final String coordinationNodePath; private final String groupName; private final RetryPolicy retryPolicy; + private final ScheduledExecutorService scheduledExecutor; - private final CoordinationSession session; + private final CoordinationSession coordinationSession; private final SemaphoreObserver semaphoreObserver; - private final ListenableContainer sessionStateListenable; + private final ListenableContainer sessionListenable; private final ListenableContainer> groupMembersListenable; - private Future acquireTask; + private final AtomicReference state = new AtomicReference<>(State.INITIAL); + private final AtomicReference> initializingTask = new AtomicReference<>(null); + private Future acquireTask = null; + /** + * Internal state + */ + private enum State { + /** Initial state before starting */ + INITIAL, + /** When start() has been called but initialization isn't complete */ + STARTING, + /** Fully operational state */ + STARTED, + /** Failed terminated state */ + FAILED, + /** Closed terminated state */ + CLOSED + } + + /** + * Creates a new GroupMembership with default settings. + * + * @param coordinationClient the coordination service client + * @param coordinationNodePath path to the coordination node + * @param groupName name of the group to join + * @throws IllegalArgumentException if any argument is invalid + */ public GroupMembership( CoordinationClient coordinationClient, String coordinationNodePath, @@ -48,28 +100,50 @@ public GroupMembership( ); } + /** + * Creates a new GroupMembership with custom settings. + * + * @param coordinationClient the coordination service client + * @param coordinationNodePath path to the coordination node + * @param groupName name of the group to join + * @param settings configuration settings + * @throws IllegalArgumentException if any argument is invalid + * @throws NullPointerException if any required argument is null + */ public GroupMembership( CoordinationClient coordinationClient, String coordinationNodePath, String groupName, GroupMembershipSettings settings ) { - this.coordinationClient = coordinationClient; - this.coordinationNodePath = coordinationNodePath; + validateConstructorArgs(coordinationClient, coordinationNodePath, groupName, settings); + this.groupName = groupName; this.retryPolicy = settings.getRetryPolicy(); + this.scheduledExecutor = settings.getScheduledExecutor(); - this.session = coordinationClient.createSession( + this.coordinationSession = coordinationClient.createSession( coordinationNodePath, CoordinationSessionSettings.newBuilder() .withRetryPolicy(retryPolicy) .build() ); - this.sessionStateListenable = new ListenableContainer<>(); - session.addStateListener(sessionStateListenable::notifyListeners); + this.sessionListenable = new ListenableContainer<>(); + coordinationSession.addStateListener(sessionState -> { + if (!state.get().equals(State.CLOSED) && (sessionState == CoordinationSession.State.LOST || + sessionState == CoordinationSession.State.CLOSED)) { + logger.error("Coordination session unexpectedly changed to {} state, group membership went FAILED", + sessionState); + stopInternal(State.FAILED); + } + if (sessionState == CoordinationSession.State.RECONNECTED) { + reconnect(); + } + sessionListenable.notifyListeners(sessionState); + }); this.semaphoreObserver = new SemaphoreObserver( - session, + coordinationSession, groupName, WatchSemaphoreMode.WATCH_OWNERS, DescribeSemaphoreMode.WITH_OWNERS, @@ -83,22 +157,153 @@ public GroupMembership( }); } + + private void validateConstructorArgs( + CoordinationClient coordinationClient, + String coordinationNodePath, + String groupName, + GroupMembershipSettings settings + ) { + Objects.requireNonNull(coordinationClient, "CoordinationClient cannot be null"); + Objects.requireNonNull(coordinationNodePath, "Coordination node path cannot be null"); + Objects.requireNonNull(groupName, "Group name cannot be null"); + Objects.requireNonNull(settings, "Settings cannot be null"); + Objects.requireNonNull(settings.getRetryPolicy(), "Retry policy cannot be null"); + Objects.requireNonNull(settings.getScheduledExecutor(), "Scheduled executor cannot be null"); + + if (groupName.isEmpty()) { + throw new IllegalArgumentException("Group name cannot be empty"); + } + if (coordinationNodePath.isEmpty()) { + throw new IllegalArgumentException("Coordination node path cannot be empty"); + } + } + + private void reconnect() { + logger.info("Attempting to reconnect group membership for group '{}'", groupName); + tryEnqueueAcquire(); + } + + /** + * Starts the group membership service. + * + *

This begins the process of joining the group and starts tracking membership. + * + * @throws IllegalStateException if already started + */ public void start() { - // TODO: correctly handle failed connection, failed semaphore and failed starts - session.connect().thenAccept(sessionStatus -> { - sessionStatus.expectSuccess("Unable to establish session"); - session.createSemaphore(groupName, MAX_GROUP_SIZE).thenAccept(semaphoreStatus -> { - semaphoreObserver.start(); - enqueueAcquire(); - }); - }); + Preconditions.checkState( + state.compareAndSet(State.INITIAL, State.STARTING), + "Group membership may be started only once" + ); + + logger.info("Starting group membership initialization for group '{}'", groupName); + + CompletableFuture sessionConnectTask = getSessionConnectRetryableTask(); + CompletableFuture semaphoreCreateTask = getSemaphoreCreateTask(); + + CompletableFuture initializingRetriedTask = sessionConnectTask + .thenCompose(connectionStatus -> { + connectionStatus.expectSuccess("Unable to establish session"); + logger.debug("Successfully connected session for group '{}'", groupName); + return semaphoreCreateTask; + }) + .thenApply(semaphoreStatus -> { + if (semaphoreStatus.isSuccess()) { + logger.info("Successfully initialized semaphore for group '{}'", groupName); + onInitializeSuccess(); + } else { + logger.error("Failed to create semaphore for group '{}': {}", groupName, semaphoreStatus); + } + semaphoreStatus.expectSuccess("Unable to create semaphore"); + return semaphoreStatus; + }).exceptionally(ex -> { + logger.error("Group membership initialization failed for group '{}'", groupName, ex); + stopInternal(State.FAILED); + return Status.of(StatusCode.CLIENT_INTERNAL_ERROR); + }); + + initializingTask.set(initializingRetriedTask); + } + + private CompletableFuture getSessionConnectRetryableTask() { + return new RetryableTask( + "groupMembership-sessionConnect-" + groupName, + coordinationSession::connect, + scheduledExecutor, + retryPolicy + ).execute(); } - private void enqueueAcquire() { - // TODO: enqueue acquire task and retry it with retry policy - acquireTask = session.acquireSemaphore(groupName, 1, Duration.ofSeconds(30)); + private CompletableFuture getSemaphoreCreateTask() { + Supplier> semaphoreCreateTaskSupplier = () -> + coordinationSession.createSemaphore(groupName, MAX_GROUP_SIZE) + .thenCompose(status -> { + if (status.getCode() == StatusCode.ALREADY_EXISTS) { + return CompletableFuture.completedFuture(Status.SUCCESS); + } + return CompletableFuture.completedFuture(status); + }); + return new RetryableTask( + "groupMembership-semaphoreCreate-" + groupName, + semaphoreCreateTaskSupplier, + scheduledExecutor, + retryPolicy + ).execute(); } + private void onInitializeSuccess() { + logger.info("Group membership initialization completed successfully for group '{}'", groupName); + state.set(State.STARTED); + semaphoreObserver.start(); + tryEnqueueAcquire(); + } + + /** + * Enqueues task if no current working task + */ + private synchronized boolean tryEnqueueAcquire() { + if (acquireTask != null) { + logger.warn("Acquire task already in progress for group '{}', skipping", groupName); + return false; + } + + logger.debug("Enqueuing new acquire task for group '{}'", groupName); + CompletableFuture acquireRetryableTask = new RetryableTask( + "groupMembership-acquireSemaphoreTask-" + groupName, + () -> coordinationSession.acquireSemaphore(groupName, 1, ACQUIRE_TIMEOUT) + .thenApply(Result::getStatus), + scheduledExecutor, + retryPolicy + ).execute(); + + acquireTask = acquireRetryableTask.whenComplete(this::finishAcquireTask); + return true; + } + + private synchronized void finishAcquireTask(Status status, @Nullable Throwable throwable) { + acquireTask = null; + + if (throwable != null) { + logger.error("Acquire task failed with exception for group '{}'", groupName, throwable); + tryEnqueueAcquire(); + return; + } + + if (status.isSuccess()) { + logger.info("Successfully acquired semaphore for group '{}'", groupName); + return; + } + + logger.warn("Failed to acquire semaphore for group '{}' with status: '{}'", groupName, status); + tryEnqueueAcquire(); + } + + /** + * Gets the current list of group members. + * + * @return list of current group members, or null if not available + */ public @Nullable List getCurrentMembers() { SemaphoreDescription cachedDescription = semaphoreObserver.getCachedData(); return mapSemaphoreDescriptionToMembersList(cachedDescription); @@ -120,19 +325,80 @@ private static GroupMember mapSessionToGroupMember(SemaphoreDescription.Session ); } + /** + * Gets a listenable for session state changes. + * + * @return observable for coordination session state changes + */ public Listenable getSessionListenable() { - return sessionStateListenable; + return sessionListenable; } + /** + * Gets a listenable for group membership changes. + * + * @return observable for group membership changes + */ public Listenable> getMembersListenable() { return groupMembersListenable; } + /** + * Closes the group membership and releases all resources. + * + *

After closing, the instance cannot be reused. + * + * @throws IOException if an I/O error occurs + */ @Override public void close() throws IOException { - session.close(); - semaphoreObserver.close(); + logger.info("Closing group membership for group '{}'", groupName); + stopInternal(State.CLOSED); + } + + /** + * Stops the service and transitions to specified termination state. + * + * @param terminationState either FAILED or CLOSED + * @return true if state was changed, false if already terminated + */ + private synchronized boolean stopInternal(State terminationState) { + State localState = state.get(); + if (localState == State.FAILED || localState == State.CLOSED) { + logger.warn("Attempted to stop already stopped group membership '{}' (current state: {})", + groupName, localState); + return false; + } + logger.info("Stopping group membership '{}' (current state: {}, target state: {})", + groupName, localState, terminationState); + + // change state + state.set(terminationState); + + // stop tasks + Future localInitializingTask = initializingTask.get(); + if (localInitializingTask != null) { + localInitializingTask.cancel(true); + initializingTask.set(null); + } + Future localAcquireTask = acquireTask; + if (localAcquireTask != null) { + localAcquireTask.cancel(true); + acquireTask = null; + } + + // Clean up resources + try { + semaphoreObserver.close(); + } catch (Exception e) { + logger.warn("Error closing semaphore observer for {}: {}", groupName, e.getMessage()); + } + try { + coordinationSession.close(); + } catch (Exception e) { + logger.warn("Error closing session for {}: {}", groupName, e.getMessage()); + } - // TODO: stop acquire task + return true; } } diff --git a/coordination/src/test/java/tech/ydb/coordination/AwaitAssert.java b/coordination/src/test/java/tech/ydb/coordination/AwaitAssert.java new file mode 100644 index 000000000..288d84617 --- /dev/null +++ b/coordination/src/test/java/tech/ydb/coordination/AwaitAssert.java @@ -0,0 +1,46 @@ +package tech.ydb.coordination; + +import java.time.Duration; +import java.util.function.BooleanSupplier; + +public class AwaitAssert { + private static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(10); + private static final Duration DEFAULT_POLL_INTERVAL = Duration.ofMillis(50); + + private Duration timeout = DEFAULT_TIMEOUT; + private Duration pollInterval = DEFAULT_POLL_INTERVAL; + + public static AwaitAssert await() { + return new AwaitAssert(); + } + + public AwaitAssert timeout(Duration timeout) { + this.timeout = timeout; + return this; + } + + public AwaitAssert pollInterval(Duration pollInterval) { + this.pollInterval = pollInterval; + return this; + } + + public void until(BooleanSupplier condition) { + long endTime = System.currentTimeMillis() + timeout.toMillis(); + + while (System.currentTimeMillis() < endTime) { + if (condition.getAsBoolean()) { + return; + } + + try { + Thread.sleep(pollInterval.toMillis()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Await interrupted", e); + } + } + + throw new RuntimeException("Condition not met within " + timeout); + } +} + diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java index c14997d55..16b888b57 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java @@ -1,12 +1,10 @@ package tech.ydb.coordination.recipes.election; import java.nio.charset.StandardCharsets; -import java.time.Duration; import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BooleanSupplier; import org.junit.AfterClass; import org.junit.Assert; @@ -15,6 +13,7 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import tech.ydb.coordination.AwaitAssert; import tech.ydb.coordination.CoordinationClient; import tech.ydb.test.junit4.GrpcTransportRule; @@ -80,8 +79,9 @@ public void takeLeadership() throws Exception { elector.start(); elector.requeue(); - await(Duration.ofSeconds(10), Duration.ofMillis(100), leadershipTaken::get); + AwaitAssert.await().until(leadershipTaken::get); Assert.assertTrue(leadershipTaken.get()); + elector.close(); } @Test @@ -98,10 +98,11 @@ public void takeLeadership() throws Exception { elector.start(); elector.requeue(); - await(Duration.ofSeconds(10), Duration.ofMillis(100), () -> leadershipCount.get() > 0); + AwaitAssert.await().until(() -> leadershipCount.get() == 1); elector.requeue(); - await(Duration.ofSeconds(10), Duration.ofMillis(100), () -> leadershipCount.get() > 1); + AwaitAssert.await().until(() -> leadershipCount.get() == 2); + elector.close(); } @Test @@ -122,7 +123,7 @@ public void takeLeadership() throws Exception { elector1.start(); elector1.requeue(); - await(Duration.ofSeconds(10), Duration.ofMillis(100), leader1Taken::get); + AwaitAssert.await().until(leader1Taken::get); // Check participants and leader List participants1 = elector1.getParticipants(); @@ -148,7 +149,7 @@ public void takeLeadership() throws Exception { elector2.start(); elector2.requeue(); - await(Duration.ofSeconds(10), Duration.ofMillis(100), leader2Taken::get); + AwaitAssert.await().until(leader2Taken::get); // Check participants and leader participants1 = elector1.getParticipants(); leader1 = elector1.getCurrentLeader(); @@ -165,24 +166,8 @@ public void takeLeadership() throws Exception { Assert.assertEquals(elector2.getCurrentLeader().get().getSessionId(), elector1.getCurrentLeader().get().getSessionId()); Assert.assertFalse(elector1.isLeader()); - } - - public static void await(Duration waitDuration, Duration checkDuration, BooleanSupplier condition) { - long timeout = System.currentTimeMillis() + waitDuration.toMillis(); - - while (System.currentTimeMillis() < timeout) { - if (condition.getAsBoolean()) { - return; - } - - try { - Thread.sleep(checkDuration.toMillis()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Await interrupted", e); - } - } - throw new RuntimeException("Condition not met within " + waitDuration); + elector1.close(); + elector2.close(); } } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java index ea5501024..ba29ea4bd 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java @@ -1,14 +1,12 @@ package tech.ydb.coordination.recipes.group; -import java.util.List; - import org.junit.AfterClass; -import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import tech.ydb.coordination.AwaitAssert; import tech.ydb.coordination.CoordinationClient; import tech.ydb.test.junit4.GrpcTransportRule; @@ -55,8 +53,13 @@ public void successTest() throws Exception { GroupMembership groupMembership = getGroupMembership(testName); groupMembership.start(); - List currentMembers = groupMembership.getCurrentMembers(); - Assert.assertEquals(1, currentMembers.size()); + + AwaitAssert.await().until(() -> { + if (groupMembership.getCurrentMembers() == null) { + return false; + } + return groupMembership.getCurrentMembers().size() == 1; + }); groupMembership.close(); } From c68d8818889ceecd6b37473f61d54606cb8a5093 Mon Sep 17 00:00:00 2001 From: wieceslaw Date: Sun, 11 May 2025 12:40:03 +0300 Subject: [PATCH 10/10] upd --- .../recipes/election/LeaderElection.java | 10 +-- .../recipes/group/GroupMembership.java | 14 +++- .../recipes/locks/InterProcessMutex.java | 3 + .../locks/ReadWriteInterProcessLock.java | 2 +- .../recipes/util/RetryableTask.java | 1 + .../CoordinationSessionBaseMockedTest.java | 10 +++ .../LeaderElectionIntegrationTest.java | 40 +++++++++- .../group/GroupMembershipIntegrationTest.java | 74 ++++++++++++++++++- .../InterProcessMutexIntegrationTest.java | 30 ++++++++ ...dWriteInterProcessLockIntegrationTest.java | 28 +++++++ .../util/SemaphoreObserverMockedTest.java | 60 +++++++++++++++ 11 files changed, 256 insertions(+), 16 deletions(-) diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java index afec97eb2..a427e1c12 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/election/LeaderElection.java @@ -11,14 +11,12 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; import com.google.common.base.Preconditions; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,8 +28,8 @@ import tech.ydb.coordination.recipes.util.Listenable; import tech.ydb.coordination.recipes.util.ListenableContainer; import tech.ydb.coordination.recipes.util.RetryableTask; -import tech.ydb.coordination.recipes.util.SessionListenableProvider; import tech.ydb.coordination.recipes.util.SemaphoreObserver; +import tech.ydb.coordination.recipes.util.SessionListenableProvider; import tech.ydb.coordination.settings.DescribeSemaphoreMode; import tech.ydb.coordination.settings.WatchSemaphoreMode; import tech.ydb.core.Status; @@ -54,10 +52,6 @@ */ public class LeaderElection implements Closeable, SessionListenableProvider { private static final Logger logger = LoggerFactory.getLogger(LeaderElection.class); - private static final ThreadFactory threadFactory = new ThreadFactoryBuilder() - .setNameFormat("ydb-leader-election-%d") - .setDaemon(true) - .build(); private static final long MAX_LEASE = 1L; private final LeaderElectionListener leaderElectionListener; @@ -148,7 +142,7 @@ public LeaderElection( this.data = data; this.leaderElectionListener = leaderElectionListener; this.scheduledExecutor = settings.getScheduledExecutor(); - this.blockingExecutor = Executors.newSingleThreadExecutor(threadFactory); + this.blockingExecutor = Executors.newSingleThreadExecutor(); this.retryPolicy = settings.getRetryPolicy(); this.coordinationSession = client.createSession(coordinationNodePath); diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java index b0368d39b..8b38876d3 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/group/GroupMembership.java @@ -16,6 +16,7 @@ import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import tech.ydb.common.retry.RetryPolicy; import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; @@ -24,6 +25,7 @@ import tech.ydb.coordination.recipes.util.ListenableContainer; import tech.ydb.coordination.recipes.util.RetryableTask; import tech.ydb.coordination.recipes.util.SemaphoreObserver; +import tech.ydb.coordination.recipes.util.SessionListenableProvider; import tech.ydb.coordination.settings.CoordinationSessionSettings; import tech.ydb.coordination.settings.DescribeSemaphoreMode; import tech.ydb.coordination.settings.WatchSemaphoreMode; @@ -44,12 +46,13 @@ * *

The implementation uses a semaphore with watch capabilities to track membership. */ -public class GroupMembership implements Closeable { +public class GroupMembership implements Closeable, SessionListenableProvider { private static final Logger logger = LoggerFactory.getLogger(GroupMembership.class); private static final long MAX_GROUP_SIZE = Long.MAX_VALUE; private static final Duration ACQUIRE_TIMEOUT = Duration.ofSeconds(30); private final String groupName; + private final byte[] data; private final RetryPolicy retryPolicy; private final ScheduledExecutorService scheduledExecutor; @@ -89,12 +92,14 @@ private enum State { public GroupMembership( CoordinationClient coordinationClient, String coordinationNodePath, - String groupName + String groupName, + byte[] data ) { this( coordinationClient, coordinationNodePath, groupName, + data, GroupMembershipSettings.newBuilder() .build() ); @@ -114,11 +119,13 @@ public GroupMembership( CoordinationClient coordinationClient, String coordinationNodePath, String groupName, + byte[] data, GroupMembershipSettings settings ) { validateConstructorArgs(coordinationClient, coordinationNodePath, groupName, settings); this.groupName = groupName; + this.data = data; this.retryPolicy = settings.getRetryPolicy(); this.scheduledExecutor = settings.getScheduledExecutor(); @@ -271,7 +278,7 @@ private synchronized boolean tryEnqueueAcquire() { logger.debug("Enqueuing new acquire task for group '{}'", groupName); CompletableFuture acquireRetryableTask = new RetryableTask( "groupMembership-acquireSemaphoreTask-" + groupName, - () -> coordinationSession.acquireSemaphore(groupName, 1, ACQUIRE_TIMEOUT) + () -> coordinationSession.acquireSemaphore(groupName, 1, data, ACQUIRE_TIMEOUT) .thenApply(Result::getStatus), scheduledExecutor, retryPolicy @@ -330,6 +337,7 @@ private static GroupMember mapSessionToGroupMember(SemaphoreDescription.Session * * @return observable for coordination session state changes */ + @Override public Listenable getSessionListenable() { return sessionListenable; } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java index 6929723fc..e1b297ee7 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/InterProcessMutex.java @@ -10,6 +10,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import tech.ydb.coordination.CoordinationClient; import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.recipes.locks.exception.LockException; @@ -96,6 +97,7 @@ public InterProcessMutex( sessionState); state.set(State.FAILED); } + logger.info("New State: " + sessionState); sessionListenable.notifyListeners(sessionState); }); @@ -191,6 +193,7 @@ public void close() { state.set(State.CLOSED); try { lockInternals.close(); + coordinationSession.close(); } catch (Exception e) { logger.warn("Error while closing lock internals '{}'", lockName, e); } diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java index 61897d404..3137f5d1b 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLock.java @@ -149,7 +149,7 @@ private enum State { CLOSED } - public InternalLock( + InternalLock( CoordinationClient client, String coordinationNodePath, String lockName, diff --git a/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java b/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java index 1e7ccf43e..0e3649e59 100644 --- a/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java +++ b/coordination/src/main/java/tech/ydb/coordination/recipes/util/RetryableTask.java @@ -7,6 +7,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import tech.ydb.common.retry.RetryPolicy; import tech.ydb.core.Status; diff --git a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java index 9143d7864..788433cb9 100644 --- a/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/CoordinationSessionBaseMockedTest.java @@ -14,6 +14,8 @@ import org.mockito.stubbing.OngoingStubbing; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.description.SemaphoreWatcher; import tech.ydb.core.Result; import tech.ydb.core.Status; import tech.ydb.core.StatusCode; @@ -211,6 +213,14 @@ public OngoingStubbing>> acquireEphemer return when(coordinationSession.acquireEphemeralSemaphore(anyString(), anyBoolean(), any(), any())); } + public OngoingStubbing>> describeSemaphore() { + return when(coordinationSession.describeSemaphore(anyString(), any())); + } + + public OngoingStubbing>> watchSemaphore() { + return when(coordinationSession.watchSemaphore(anyString(), any(), any())); + } + public void connecting() { changeState(CoordinationSession.State.CONNECTING); } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java index 16b888b57..6d4857fd3 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/election/LeaderElectionIntegrationTest.java @@ -84,6 +84,39 @@ public void takeLeadership() throws Exception { elector.close(); } + @Test + public void interruptLeadership_ThenStops() throws Exception { + AtomicBoolean leadershipTaken = new AtomicBoolean(false); + AtomicBoolean interrupted = new AtomicBoolean(false); + + String testName = "interruptLeadership_ThenStops"; + LeaderElection elector = getLeaderElector(testName, new LeaderElectionListener() { + @Override + public void takeLeadership() throws Exception { + try { + logger.debug("Leadership is taken"); + leadershipTaken.set(true); + Thread.sleep(10000); + } catch (InterruptedException e) { + interrupted.set(true); + logger.debug("Leadership is interrupted"); + } + } + }); + elector.start(); + elector.requeue(); + + AwaitAssert.await().until(leadershipTaken::get); + Assert.assertTrue(leadershipTaken.get()); + + elector.interruptLeadership(); + AwaitAssert.await().until(interrupted::get); + Assert.assertFalse(elector.isLeader()); + + elector.close(); + } + + @Test public void shouldCallTakeLeadershipAgainAfterRequeue() throws Exception { AtomicInteger leadershipCount = new AtomicInteger(0); @@ -133,6 +166,8 @@ public void takeLeadership() throws Exception { Assert.assertEquals(1, participants1.size()); Assert.assertTrue(leader1.isPresent()); + Assert.assertTrue(leader1.get().isLeader()); + Assert.assertArrayEquals(leader1.get().getData(), testName.getBytes(StandardCharsets.UTF_8)); Assert.assertEquals(participants1.get(0).getSessionId(), leader1.get().getSessionId()); // Add second leader @@ -146,8 +181,8 @@ public void takeLeadership() throws Exception { logger.info("Leadership 2 ended"); } }); + elector2.autoRequeue(); elector2.start(); - elector2.requeue(); AwaitAssert.await().until(leader2Taken::get); // Check participants and leader @@ -161,6 +196,9 @@ public void takeLeadership() throws Exception { logger.info("current leader 2 {}", leader2); logger.info("current participants 2 {}", participants2); + Assert.assertEquals(participants1, participants2); + Assert.assertEquals(leader1.hashCode(), leader2.hashCode()); + Assert.assertTrue(leader2Taken.get()); Assert.assertTrue(elector2.isLeader()); Assert.assertEquals(elector2.getCurrentLeader().get().getSessionId(), diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java index ba29ea4bd..c4b414981 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/group/GroupMembershipIntegrationTest.java @@ -1,11 +1,16 @@ package tech.ydb.coordination.recipes.group; +import java.util.List; +import java.util.concurrent.Executors; + import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import tech.ydb.common.retry.RetryForever; import tech.ydb.coordination.AwaitAssert; import tech.ydb.coordination.CoordinationClient; import tech.ydb.test.junit4.GrpcTransportRule; @@ -29,12 +34,13 @@ public static void clean() { } private GroupMembership getGroupMembership(String testName) { - return getGroupMembership(testName, testName); + return getGroupMembership(testName, testName, testName.getBytes()); } private GroupMembership getGroupMembership( String coordinationNodePath, - String groupName + String groupName, + byte[] data ) { client.createNode(coordinationNodePath).join().expectSuccess( "cannot create coordination node on path: " + coordinationNodePath @@ -42,7 +48,26 @@ private GroupMembership getGroupMembership( return new GroupMembership( client, coordinationNodePath, - groupName + groupName, + data + ); + } + + private GroupMembership getGroupMembershipCustom( + String coordinationNodePath, + String groupName, + byte[] data, + GroupMembershipSettings settings + ) { + client.createNode(coordinationNodePath).join().expectSuccess( + "cannot create coordination node on path: " + coordinationNodePath + ); + return new GroupMembership( + client, + coordinationNodePath, + groupName, + data, + settings ); } @@ -51,6 +76,39 @@ public void successTest() throws Exception { String testName = "successTest"; GroupMembership groupMembership = getGroupMembership(testName); + + groupMembership.getSessionListenable().addListener( + state -> logger.info("State change: " + state) + ); + groupMembership.getMembersListenable().addListener( + groupMembers -> logger.info("Members change: " + groupMembers) + ); + + groupMembership.start(); + + AwaitAssert.await().until(() -> { + if (groupMembership.getCurrentMembers() == null) { + return false; + } + return groupMembership.getCurrentMembers().size() == 1; + }); + + groupMembership.close(); + } + + @Test + public void everyTest() throws Exception { + String testName = "everyTest"; + + GroupMembership groupMembership = getGroupMembershipCustom( + testName, + testName, + testName.getBytes(), + GroupMembershipSettings.newBuilder() + .withRetryPolicy(new RetryForever(100)) + .withScheduledExecutor(Executors.newSingleThreadScheduledExecutor()) + .build() + ); groupMembership.start(); @@ -61,6 +119,16 @@ public void successTest() throws Exception { return groupMembership.getCurrentMembers().size() == 1; }); + List currentMembers = groupMembership.getCurrentMembers(); + GroupMember groupMember1 = currentMembers.get(0); + logger.info(groupMember1.toString()); + + Assert.assertEquals(1L, groupMember1.getSessionId()); + Assert.assertArrayEquals(groupMember1.getData(), testName.getBytes()); + GroupMember groupMember2 = new GroupMember(1L, testName.getBytes()); + Assert.assertEquals(groupMember1, groupMember2); + Assert.assertEquals(groupMember1.hashCode(), groupMember2.hashCode()); + groupMembership.close(); } diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java index bedd8eb35..b8cc068b9 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/InterProcessMutexIntegrationTest.java @@ -9,6 +9,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.stream.Collectors; import org.junit.AfterClass; @@ -19,6 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; import tech.ydb.coordination.recipes.locks.exception.LockAlreadyAcquiredException; import tech.ydb.test.junit4.GrpcTransportRule; @@ -74,6 +76,34 @@ public void simpleLockTest() throws Exception { lock.release(); } + /** + * Asserts that code does not throw any exceptions + */ + @Test(timeout = 10000) + public void sessionListenerTest() throws Exception { + InterProcessMutex lock = getInterProcessMutex(); + + Consumer syncListener = state -> logger.info("Recieved sync state change: " + state); + Consumer asyncListener = state -> logger.info("Recieved async state change: " + state); + + lock.getSessionListenable().addListener(syncListener); + // try add listener twice + lock.getSessionListenable().addListener(syncListener); + + lock.getSessionListenable().addListener(asyncListener, Executors.newSingleThreadExecutor()); + // try add listener twice + lock.getSessionListenable().addListener(asyncListener, Executors.newSingleThreadExecutor()); + + lock.acquire(); + Assert.assertTrue(lock.isAcquiredInThisProcess()); + Thread.sleep(100); + lock.release(); + lock.close(); + + lock.getSessionListenable().removeListener(syncListener); + lock.getSessionListenable().removeListener(asyncListener); + } + /** * Asserts that code does not throw any exceptions */ diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java index 6459eeb57..be3717451 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/locks/ReadWriteInterProcessLockIntegrationTest.java @@ -10,6 +10,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.stream.Collectors; import org.junit.AfterClass; @@ -20,6 +21,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import tech.ydb.coordination.CoordinationClient; +import tech.ydb.coordination.CoordinationSession; import tech.ydb.test.junit4.GrpcTransportRule; public class ReadWriteInterProcessLockIntegrationTest { @@ -47,10 +49,16 @@ public static void clean() { public void simpleWriteLockTest() throws Exception { ReadWriteInterProcessLock rwLock = getReadWriteLock(); + Consumer listener = state -> logger.info("New state: " + state); + rwLock.writeLock().getSessionListenable().addListener(listener); + rwLock.writeLock().acquire(); Assert.assertTrue(rwLock.writeLock().isAcquiredInThisProcess()); Thread.sleep(100); rwLock.writeLock().release(); + + rwLock.close(); + rwLock.writeLock().getSessionListenable().removeListener(listener); } /** @@ -64,6 +72,8 @@ public void simpleReadLockTest() throws Exception { Assert.assertTrue(rwLock.readLock().isAcquiredInThisProcess()); Thread.sleep(100); rwLock.readLock().release(); + + rwLock.close(); } /** @@ -81,6 +91,9 @@ public void writeLockExclusivityTest() throws Exception { Assert.assertFalse(lock2.writeLock().acquire(Duration.ofMillis(100))); Assert.assertFalse(lock2.writeLock().isAcquiredInThisProcess()); + + lock1.close(); + lock2.close(); } /** @@ -98,6 +111,9 @@ public void readLockSharedAccessTest() throws Exception { Assert.assertTrue(lock2.readLock().acquire(Duration.ofMillis(100))); Assert.assertTrue(lock2.readLock().isAcquiredInThisProcess()); + + lock1.close(); + lock2.close(); } @@ -116,6 +132,9 @@ public void writeLockBlockedByReadLockTest() throws Exception { Assert.assertFalse(lock2.writeLock().acquire(Duration.ofMillis(100))); Assert.assertFalse(lock2.writeLock().isAcquiredInThisProcess()); + + lock1.close(); + lock2.close(); } /** @@ -133,6 +152,9 @@ public void readLockBlockedByWriteLockTest() throws Exception { Assert.assertFalse(lock2.readLock().acquire(Duration.ofMillis(100))); Assert.assertFalse(lock2.readLock().isAcquiredInThisProcess()); + + lock1.close(); + lock2.close(); } /** @@ -254,6 +276,10 @@ public void mixedReadWriteLockTest() throws Exception { // Cleanup executor.shutdown(); + + lock1.close(); + lock2.close(); + lock3.close(); } /** @@ -284,6 +310,8 @@ public void reacquisitionTest() throws Exception { Assert.assertTrue(rwLock.readLock().acquire(Duration.ofMillis(100))); Assert.assertTrue(rwLock.readLock().isAcquiredInThisProcess()); rwLock.readLock().release(); + + rwLock.close(); } private ReadWriteInterProcessLock getReadWriteLock() { diff --git a/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java b/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java index f0425cda8..bd3a6645d 100644 --- a/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java +++ b/coordination/src/test/java/tech/ydb/coordination/recipes/util/SemaphoreObserverMockedTest.java @@ -1,12 +1,19 @@ package tech.ydb.coordination.recipes.util; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import org.junit.Test; +import org.mockito.Mockito; import tech.ydb.common.retry.RetryForever; import tech.ydb.coordination.CoordinationSessionBaseMockedTest; +import tech.ydb.coordination.description.SemaphoreChangedEvent; +import tech.ydb.coordination.description.SemaphoreDescription; +import tech.ydb.coordination.description.SemaphoreWatcher; import tech.ydb.coordination.settings.DescribeSemaphoreMode; import tech.ydb.coordination.settings.WatchSemaphoreMode; +import tech.ydb.core.Result; +import tech.ydb.core.Status; public class SemaphoreObserverMockedTest extends CoordinationSessionBaseMockedTest { @@ -15,6 +22,58 @@ public void successTest() { SessionMock sessionMock = getSessionMock(); sessionMock.connect() .then(successConnect()); + sessionMock.describeSemaphore() + .thenReturn(CompletableFuture.completedFuture(Mockito.mock(Result.class))); + + Result watchResultMock = Mockito.mock(Result.class); + Mockito.when(watchResultMock.isSuccess()).thenReturn(true); + Mockito.when(watchResultMock.getStatus()).thenReturn(Status.SUCCESS); + SemaphoreWatcher semaphoreWatcher = Mockito.mock(SemaphoreWatcher.class); + Mockito.when(semaphoreWatcher.getDescription()).thenReturn(Mockito.mock(SemaphoreDescription.class)); + Result watchResultEventMock = Mockito.mock(Result.class); + CompletableFuture> watchedEvent = + CompletableFuture.completedFuture(watchResultEventMock); + Mockito.when(semaphoreWatcher.getChangedFuture()).thenReturn(watchedEvent); + Mockito.when(watchResultMock.getValue()).thenReturn(semaphoreWatcher); + + sessionMock.watchSemaphore() + .thenReturn(CompletableFuture.completedFuture(watchResultMock)); + + SemaphoreObserver observer = new SemaphoreObserver( + getCoordinationSession(), + "observable_semaphore", + WatchSemaphoreMode.WATCH_DATA_AND_OWNERS, + DescribeSemaphoreMode.WITH_OWNERS_AND_WAITERS, + new RetryForever(100), + Executors.newSingleThreadScheduledExecutor() + ); + observer.start(); + sessionMock.connected(); + observer.getCachedData(); + observer.close(); + } + + @Test + public void start_alreadyStarted_Error() { + SessionMock sessionMock = getSessionMock(); + sessionMock.connect() + .then(successConnect()); + sessionMock.describeSemaphore() + .thenReturn(CompletableFuture.completedFuture(Mockito.mock(Result.class))); + + Result watchResultMock = Mockito.mock(Result.class); + Mockito.when(watchResultMock.isSuccess()).thenReturn(true); + Mockito.when(watchResultMock.getStatus()).thenReturn(Status.SUCCESS); + SemaphoreWatcher semaphoreWatcher = Mockito.mock(SemaphoreWatcher.class); + Mockito.when(semaphoreWatcher.getDescription()).thenReturn(Mockito.mock(SemaphoreDescription.class)); + Result watchResultEventMock = Mockito.mock(Result.class); + CompletableFuture> watchedEvent = + CompletableFuture.completedFuture(watchResultEventMock); + Mockito.when(semaphoreWatcher.getChangedFuture()).thenReturn(watchedEvent); + Mockito.when(watchResultMock.getValue()).thenReturn(semaphoreWatcher); + + sessionMock.watchSemaphore() + .thenReturn(CompletableFuture.completedFuture(watchResultMock)); SemaphoreObserver observer = new SemaphoreObserver( getCoordinationSession(), @@ -27,6 +86,7 @@ public void successTest() { observer.start(); sessionMock.connected(); observer.getCachedData(); + observer.close(); } }