From 4a062af92d89a8659fd017f189819336eb8c0072 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Fri, 3 May 2024 16:16:16 -0400 Subject: [PATCH 01/17] Fate reservations moved out of memory - Reservations for MetaFateStore were moved out of memory into ZooKeeper - Reservations for UserFateStore were moved out of memory into the Accumulo Fate table - Added test MultipleStoresIT This commit is one part needed for having Fate be distributed --- .../accumulo/core/fate/AbstractFateStore.java | 219 +++++----- .../apache/accumulo/core/fate/FateStore.java | 3 +- .../accumulo/core/fate/MetaFateStore.java | 185 +++++++-- .../accumulo/core/fate/user/FateMutator.java | 9 + .../core/fate/user/FateMutatorImpl.java | 42 ++ .../core/fate/user/UserFateStore.java | 67 ++- .../core/fate/user/schema/FateSchema.java | 3 + .../apache/accumulo/server/util/Admin.java | 5 +- .../org/apache/accumulo/manager/Manager.java | 5 +- .../manager/metrics/fate/FateMetrics.java | 5 +- .../manager/upgrade/UpgradeCoordinator.java | 7 +- .../compaction/ExternalCompaction_1_IT.java | 13 +- .../accumulo/test/fate/MultipleStoresIT.java | 384 ++++++++++++++++++ .../accumulo/test/fate/meta/MetaFateIT.java | 9 +- .../test/fate/meta/MetaFateOpsCommandsIT.java | 4 +- .../test/fate/meta/MetaFateStoreFateIT.java | 28 +- .../test/fate/user/UserFateStoreIT.java | 2 +- .../test/functional/FateConcurrencyIT.java | 13 +- .../test/functional/FunctionalTestUtils.java | 7 +- 19 files changed, 820 insertions(+), 190 deletions(-) create mode 100644 test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 0bec78d1963..13b7f2c7ed8 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -28,22 +28,24 @@ import java.io.Serializable; import java.io.UncheckedIOException; import java.time.Duration; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; -import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.stream.Stream; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.time.NanoTime; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,17 +70,16 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { } }; - protected final Set reserved; protected final Map deferred; private final int maxDeferred; private final AtomicBoolean deferredOverflow = new AtomicBoolean(); private final FateIdGenerator fateIdGenerator; - // This is incremented each time a transaction was unreserved that was non new - protected final SignalCount unreservedNonNewCount = new SignalCount(); - // This is incremented each time a transaction is unreserved that was runnable - protected final SignalCount unreservedRunnableCount = new SignalCount(); + private final SignalCount unreservedRunnableCount = new SignalCount(); + + // Keeps track of the number of concurrent callers to waitForStatusChange() + private final AtomicInteger concurrentStatusChangeCallers = new AtomicInteger(0); public AbstractFateStore() { this(DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); @@ -87,8 +88,7 @@ public AbstractFateStore() { public AbstractFateStore(int maxDeferred, FateIdGenerator fateIdGenerator) { this.maxDeferred = maxDeferred; this.fateIdGenerator = Objects.requireNonNull(fateIdGenerator); - this.reserved = new HashSet<>(); - this.deferred = new HashMap<>(); + this.deferred = Collections.synchronizedMap(new HashMap<>()); } public static byte[] serialize(Object o) { @@ -115,38 +115,26 @@ public static Object deserialize(byte[] ser) { } } - /** - * Attempt to reserve the fate transaction. - * - * @param fateId The FateId - * @return An Optional containing the FateTxStore if the transaction was successfully reserved, or - * an empty Optional if the transaction was already reserved. - */ - @Override - public Optional> tryReserve(FateId fateId) { - synchronized (this) { - if (!reserved.contains(fateId)) { - return Optional.of(reserve(fateId)); - } - return Optional.empty(); - } - } - @Override public FateTxStore reserve(FateId fateId) { - synchronized (AbstractFateStore.this) { - while (reserved.contains(fateId)) { - try { - AbstractFateStore.this.wait(100); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IllegalStateException(e); - } + Preconditions.checkState(!_getStatus(fateId).equals(TStatus.UNKNOWN), + "Attempted to reserve a tx that does not exist: " + fateId); + var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25)) + .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5) + .logInterval(Duration.ofMinutes(3)).createRetry(); + Optional> reserveAttempt = tryReserve(fateId); + while (reserveAttempt.isEmpty()) { + try { + retry.waitForNextAttempt(log, "Attempting to reserve " + fateId); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalArgumentException(e); } - - reserved.add(fateId); - return newFateTxStore(fateId, true); + reserveAttempt = tryReserve(fateId); } + retry.logCompletion(log, "Attempting to reserve " + fateId); + + return reserveAttempt.orElseThrow(); } @Override @@ -161,18 +149,16 @@ public void runnable(AtomicBoolean keepWaiting, Consumer idConsumer) { try (Stream transactions = getTransactions()) { transactions.filter(fateIdStatus -> isRunnable(fateIdStatus.getStatus())) .map(FateIdStatus::getFateId).filter(fateId -> { - synchronized (AbstractFateStore.this) { - var deferredTime = deferred.get(fateId); - if (deferredTime != null) { - if (deferredTime.elapsed().isNegative()) { - // negative elapsed time indicates the deferral time is in the future - return false; - } else { - deferred.remove(fateId); - } + var deferredTime = deferred.get(fateId); + if (deferredTime != null) { + if (deferredTime.elapsed().isNegative()) { + // negative elapsed time indicates the deferral time is in the future + return false; + } else { + deferred.remove(fateId); } - return !reserved.contains(fateId); } + return !isReserved(fateId); }).forEach(fateId -> { seen.incrementAndGet(); idConsumer.accept(fateId); @@ -218,7 +204,7 @@ public Stream list() { @Override public ReadOnlyFateTxStore read(FateId fateId) { - return newFateTxStore(fateId, false); + return newUnreservedFateTxStore(fateId); } protected boolean isRunnable(TStatus status) { @@ -249,13 +235,11 @@ public int getDeferredCount() { // This method is primarily used right now for unit testing but // if this synchronization becomes an issue we could add an atomic // counter instead to track it separately so we don't need to lock - synchronized (AbstractFateStore.this) { - return deferred.size(); - } + return deferred.size(); } private Optional create(FateKey fateKey) { - FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); + FateId fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); try { create(fateId, fateKey); @@ -286,7 +270,13 @@ private Optional create(FateKey fateKey) { @Override public Optional> createAndReserve(FateKey fateKey) { - FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); + // TODO 4131 not confident about this new implementation of createAndReserve. + // Previously, you could reserve before creation, but with the new impl of reservations + // being stored in ZK (MetaFateStore) and the Accumulo Fate table (UserFateStore), creation + // is needed before reservation. + // TODO 4131 the comments in this method also need to be updated. + // Will wait until after review for this method + FateId fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); final Optional> txStore; // First make sure we can reserve in memory the fateId, if not @@ -296,34 +286,22 @@ public Optional> createAndReserve(FateKey fateKey) { // This will create the FateTxStore before creation but this object // is not exposed until after creation is finished so there should not // be any errors. - final Optional> reservedTxStore; - synchronized (this) { - reservedTxStore = tryReserve(fateId); - } // If present we were able to reserve so try and create - if (reservedTxStore.isPresent()) { + if (!isReserved(fateId)) { try { var fateIdFromCreate = create(fateKey); if (fateIdFromCreate.isPresent()) { Preconditions.checkState(fateId.equals(fateIdFromCreate.orElseThrow()), "Transaction creation returned unexpected %s, expected %s", fateIdFromCreate, fateId); - txStore = reservedTxStore; + txStore = tryReserve(fateId); } else { // We already exist in a non-new state then un-reserve and an empty // Optional will be returned. This is expected to happen when the // system is busy and operations are not running, and we keep seeding them - synchronized (this) { - reserved.remove(fateId); - } txStore = Optional.empty(); } } catch (Exception e) { - // Clean up the reservation if the creation failed - // And then throw error - synchronized (this) { - reserved.remove(fateId); - } if (e instanceof IllegalStateException) { throw e; } else { @@ -349,91 +327,104 @@ public Optional> createAndReserve(FateKey fateKey) { protected abstract Optional getKey(FateId fateId); - protected abstract FateTxStore newFateTxStore(FateId fateId, boolean isReserved); + protected abstract FateTxStore newUnreservedFateTxStore(FateId fateId); + + protected abstract boolean isReserved(FateId fateId); - protected abstract FateInstanceType getInstanceType(); + // TODO 4131 is public fine for this? Public for tests + public abstract List getReservedTxns(); protected abstract class AbstractFateTxStoreImpl implements FateTxStore { protected final FateId fateId; - protected final boolean isReserved; + protected boolean deleted; protected TStatus observedStatus = null; - protected AbstractFateTxStoreImpl(FateId fateId, boolean isReserved) { + protected AbstractFateTxStoreImpl(FateId fateId) { this.fateId = fateId; - this.isReserved = isReserved; + this.deleted = false; } + protected abstract boolean isReserved(); + @Override public TStatus waitForStatusChange(EnumSet expected) { - Preconditions.checkState(!isReserved, - "Attempted to wait for status change while reserved " + fateId); - while (true) { + Preconditions.checkState(!isReserved(), + "Attempted to wait for status change while reserved: " + fateId); + verifyReserved(false); + + int currNumCallers = concurrentStatusChangeCallers.incrementAndGet(); + // TODO 4131 + // TODO make the max time a function of the number of concurrent callers, as the number of + // concurrent callers increases then increase the max wait time + // TODO could support signaling within this instance for known events + // TODO made the maxWait low so this would be responsive... that may put a lot of load in the + // case there are lots of things waiting... + // Made maxWait = num of curr callers + var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25)) + .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(currNumCallers)) + .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry(); - long countBefore = unreservedNonNewCount.getCount(); + while (true) { TStatus status = _getStatus(fateId); if (expected.contains(status)) { + retry.logCompletion(log, "Waiting on status change for " + fateId + " expected:" + + expected + " status:" + status); + concurrentStatusChangeCallers.decrementAndGet(); return status; } - unreservedNonNewCount.waitFor(count -> count != countBefore, 1000, () -> true); + try { + retry.waitForNextAttempt(log, "Waiting on status change for " + fateId + " expected:" + + expected + " status:" + status); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + concurrentStatusChangeCallers.decrementAndGet(); + throw new IllegalStateException(e); + } } } @Override public void unreserve(long deferTime, TimeUnit timeUnit) { - Duration deferDuration = Duration.of(deferTime, timeUnit.toChronoUnit()); + Preconditions.checkState(isReserved(), + "Attempted to unreserve a transaction that was not reserved: " + fateId); + Duration deferDuration = Duration.of(deferTime, timeUnit.toChronoUnit()); if (deferDuration.isNegative()) { throw new IllegalArgumentException("deferTime < 0 : " + deferTime); } - synchronized (AbstractFateStore.this) { - if (!reserved.remove(fateId)) { - throw new IllegalStateException("Tried to unreserve id that was not reserved " + fateId); - } - - // notify any threads waiting to reserve - AbstractFateStore.this.notifyAll(); - - // If deferred map has overflowed then skip adding to the deferred map - // and clear the map and set the flag. This will cause the next execution - // of runnable to process all the transactions and to not defer as we - // have a large backlog and want to make progress - if (deferDuration.compareTo(Duration.ZERO) > 0 && !deferredOverflow.get()) { - if (deferred.size() >= maxDeferred) { - log.info( - "Deferred map overflowed with size {}, clearing and setting deferredOverflow to true", - deferred.size()); - deferredOverflow.set(true); - deferred.clear(); - } else { - deferred.put(fateId, NanoTime.nowPlus(deferDuration)); - } + // If deferred map has overflowed then skip adding to the deferred map + // and clear the map and set the flag. This will cause the next execution + // of runnable to process all the transactions and to not defer as we + // have a large backlog and want to make progress + if (deferDuration.compareTo(Duration.ZERO) > 0 && !isDeferredOverflow()) { + if (deferred.size() >= maxDeferred) { + log.info( + "Deferred map overflowed with size {}, clearing and setting deferredOverflow to true", + deferred.size()); + deferredOverflow.set(true); + deferred.clear(); + } else { + deferred.put(fateId, NanoTime.nowPlus(deferDuration)); } } + unreserve(); + if (observedStatus != null && isRunnable(observedStatus)) { unreservedRunnableCount.increment(); } - - if (observedStatus != TStatus.NEW) { - unreservedNonNewCount.increment(); - } } - protected void verifyReserved(boolean isWrite) { - if (!isReserved && isWrite) { - throw new IllegalStateException("Attempted write on unreserved FATE transaction."); - } + protected abstract void unreserve(); - if (isReserved) { - synchronized (AbstractFateStore.this) { - if (!reserved.contains(fateId)) { - throw new IllegalStateException("Tried to operate on unreserved transaction " + fateId); - } - } + protected void verifyReserved(boolean isWrite) { + if (!isReserved() && isWrite) { + throw new IllegalStateException( + "Attempted write on unreserved FATE transaction: " + fateId); } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 088e5025229..f93113112a3 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -111,7 +111,8 @@ interface FateTxStore extends ReadOnlyFateTxStore { * Attempt to reserve the fate transaction. * * @param fateId The FateId - * @return true if reserved by this call, false if already reserved + * @return An Optional containing the {@link FateTxStore} if the transaction was successfully + * reserved, or an empty Optional if the transaction was not able to be reserved. */ Optional> tryReserve(FateId fateId); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index d8cecd6a473..612a60d6fc8 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -33,10 +33,13 @@ import java.util.Optional; import java.util.UUID; import java.util.function.Supplier; +import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.util.Pair; @@ -53,29 +56,32 @@ //TODO use zoocache? - ACCUMULO-1297 //TODO handle zookeeper being down gracefully - ACCUMULO-1297 - +// TODO 4131 noticed this class is not in the fate.zookeeper package. Should it be? public class MetaFateStore extends AbstractFateStore { private static final Logger log = LoggerFactory.getLogger(MetaFateStore.class); private static final FateInstanceType fateInstanceType = FateInstanceType.META; private String path; private ZooReaderWriter zk; + // The ZooKeeper lock for the process that's running this store instance + private ZooUtil.LockID lockID; private String getTXPath(FateId fateId) { return path + "/tx_" + fateId.getTxUUIDStr(); } - public MetaFateStore(String path, ZooReaderWriter zk) + public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID) throws KeeperException, InterruptedException { - this(path, zk, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + this(path, zk, lockID, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } @VisibleForTesting - public MetaFateStore(String path, ZooReaderWriter zk, int maxDeferred, + public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID, int maxDeferred, FateIdGenerator fateIdGenerator) throws KeeperException, InterruptedException { super(maxDeferred, fateIdGenerator); this.path = path; this.zk = zk; + this.lockID = lockID; zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); } @@ -90,7 +96,7 @@ public FateId create() { while (true) { try { FateId fateId = FateId.from(fateInstanceType, UUID.randomUUID()); - zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW).serialize(), + zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, "", "").serialize(), NodeExistsPolicy.FAIL); return fateId; } catch (NodeExistsException nee) { @@ -104,13 +110,72 @@ public FateId create() { @Override protected void create(FateId fateId, FateKey key) { try { - zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, key).serialize(), + zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, "", "", key).serialize(), NodeExistsPolicy.FAIL); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } } + @Override + public Optional> tryReserve(FateId fateId) { + // uniquely identify this attempt to reserve the fate operation data + UUID uuid = UUID.randomUUID(); + + try { + byte[] newSerNodeVal = zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { + NodeValue currNodeVal = new NodeValue(currSerNodeVal); + // The uuid handles the case where there was a ZK server fault and the write for this thread + // went through but that was not acknowledged, and we are reading our own write for 2nd + // time. + if (!currNodeVal.isReserved() || currNodeVal.uuid.equals(uuid.toString())) { + FateKey currFateKey = currNodeVal.fateKey.orElse(null); + // Add the lock and uuid to the node to reserve + return new NodeValue(currNodeVal.status, lockID.serialize(""), uuid.toString(), + currFateKey).serialize(); + } else { + // This will not change the value to null but will return null + return null; + } + }); + if (newSerNodeVal != null) { + return Optional.of(new FateTxStoreImpl(fateId, uuid)); + } else { + return Optional.empty(); + } + } catch (InterruptedException | KeeperException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } + } + + @Override + protected boolean isReserved(FateId fateId) { + boolean isReserved; + try { + isReserved = getNode(fateId).isReserved(); + } catch (Exception e) { + // Exception thrown, so node doesn't exist, so it is not reserved + isReserved = false; + } + return isReserved; + } + + // TODO 4131 is public fine for this? Public for tests + @Override + public List getReservedTxns() { + try { + return zk.getChildren(path).stream().filter(strTxId -> { + String txUUIDStr = strTxId.split("_")[1]; + return isReserved(FateId.from(fateInstanceType, txUUIDStr)); + }).map(strTxId -> { + String txUUIDStr = strTxId.split("_")[1]; + return FateId.from(fateInstanceType, txUUIDStr); + }).collect(Collectors.toList()); + } catch (KeeperException | InterruptedException e) { + throw new RuntimeException(e); + } + } + @Override protected Pair> getStatusAndKey(FateId fateId) { final NodeValue node = getNode(fateId); @@ -123,9 +188,21 @@ public FateInstanceType type() { } private class FateTxStoreImpl extends AbstractFateTxStoreImpl { + private UUID reservationUUID; - private FateTxStoreImpl(FateId fateId, boolean isReserved) { - super(fateId, isReserved); + private FateTxStoreImpl(FateId fateId) { + super(fateId); + this.reservationUUID = null; + } + + private FateTxStoreImpl(FateId fateId, UUID reservationUUID) { + super(fateId); + this.reservationUUID = Objects.requireNonNull(reservationUUID); + } + + @Override + protected boolean isReserved() { + return reservationUUID != null; } private static final int RETRIES = 10; @@ -175,7 +252,7 @@ private String findTop(String txpath) throws KeeperException, InterruptedExcepti } } - if (max.equals("")) { + if (max.isEmpty()) { return null; } @@ -222,9 +299,14 @@ public void setStatus(TStatus status) { verifyReserved(true); try { - zk.putPersistentData(getTXPath(fateId), new NodeValue(status).serialize(), - NodeExistsPolicy.OVERWRITE); - } catch (KeeperException | InterruptedException e) { + zk.mutateExisting(getTXPath(fateId), currSerializedData -> { + NodeValue currNodeVal = new NodeValue(currSerializedData); + FateKey currFateKey = currNodeVal.fateKey.orElse(null); + NodeValue newNodeValue = + new NodeValue(status, currNodeVal.lockID, currNodeVal.uuid, currFateKey); + return newNodeValue.serialize(); + }); + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { throw new IllegalStateException(e); } @@ -237,6 +319,7 @@ public void delete() { try { zk.recursiveDelete(getTXPath(fateId), NodeMissingPolicy.SKIP); + this.deleted = true; } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -313,6 +396,29 @@ public List> getStack() { return dops; } } + + @Override + protected void unreserve() { + try { + if (!this.deleted) { + zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { + NodeValue currNodeVal = new NodeValue(currSerNodeVal); + FateKey currFateKey = currNodeVal.fateKey.orElse(null); + if (currNodeVal.uuid.equals(reservationUUID.toString())) { + // Remove the lock and uuid from the NodeValue to unreserve + return new NodeValue(currNodeVal.status, "", "", currFateKey).serialize(); + } else { + // possible this is running a 2nd time in zk server fault conditions and its first + // write went through + return null; + } + }); + } + this.reservationUUID = null; + } catch (InterruptedException | KeeperException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } + } } private Serializable getTransactionInfo(TxInfo txInfo, FateId fateId) { @@ -339,20 +445,15 @@ private NodeValue getNode(FateId fateId) { try { return new NodeValue(zk.getData(getTXPath(fateId))); } catch (NoNodeException nne) { - return new NodeValue(TStatus.UNKNOWN); + return new NodeValue(TStatus.UNKNOWN, "", ""); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } } @Override - protected FateTxStore newFateTxStore(FateId fateId, boolean isReserved) { - return new FateTxStoreImpl(fateId, isReserved); - } - - @Override - protected FateInstanceType getInstanceType() { - return fateInstanceType; + protected FateTxStore newUnreservedFateTxStore(FateId fateId) { + return new FateTxStoreImpl(fateId); } @Override @@ -385,23 +486,34 @@ public Stream list(FateKey.FateKeyType type) { protected static class NodeValue { final TStatus status; final Optional fateKey; + final String lockID; + final String uuid; - private NodeValue(byte[] serialized) { + private NodeValue(byte[] serializedData) { try (DataInputBuffer buffer = new DataInputBuffer()) { - buffer.reset(serialized, serialized.length); - this.status = TStatus.valueOf(buffer.readUTF()); + buffer.reset(serializedData, serializedData.length); + TStatus tempStatus = TStatus.valueOf(buffer.readUTF()); + String tempLockID = buffer.readUTF(); + String tempUUID = buffer.readUTF(); + validateLockAndUUID(tempLockID, tempUUID); + this.status = tempStatus; + this.lockID = tempLockID; + this.uuid = tempUUID; this.fateKey = deserializeFateKey(buffer); } catch (IOException e) { throw new UncheckedIOException(e); } } - private NodeValue(TStatus status) { - this(status, null); + private NodeValue(TStatus status, String lockID, String uuid) { + this(status, lockID, uuid, null); } - private NodeValue(TStatus status, FateKey fateKey) { - this.status = Objects.requireNonNull(status); + private NodeValue(TStatus status, String lockID, String uuid, FateKey fateKey) { + validateLockAndUUID(lockID, uuid); + this.status = status; + this.lockID = lockID; + this.uuid = uuid; this.fateKey = Optional.ofNullable(fateKey); } @@ -417,10 +529,12 @@ byte[] serialize() { try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos)) { dos.writeUTF(status.name()); + dos.writeUTF(lockID); + dos.writeUTF(uuid); if (fateKey.isPresent()) { - byte[] serialized = fateKey.orElseThrow().getSerialized(); - dos.writeInt(serialized.length); - dos.write(serialized); + byte[] serializedFateKey = fateKey.orElseThrow().getSerialized(); + dos.writeInt(serializedFateKey.length); + dos.write(serializedFateKey); } else { dos.writeInt(0); } @@ -431,5 +545,16 @@ byte[] serialize() { } } + public boolean isReserved() { + return !lockID.isEmpty(); + } + + private void validateLockAndUUID(String lockID, String uuid) { + // TODO 4131 potentially need further validation? + if (!((lockID.isEmpty() && uuid.isEmpty()) || (!lockID.isEmpty() && !uuid.isEmpty()))) { + throw new IllegalArgumentException( + "One but not both of lock = '" + lockID + "' and uuid = '" + uuid + "' are empty"); + } + } } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java index b57ffae68f1..8042df117cb 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.fate.user; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; @@ -31,6 +32,14 @@ public interface FateMutator { FateMutator putCreateTime(long ctime); + FateMutator putReservedTx(FateId fateId); + + FateMutator putUnreserveTx(FateId fateId); + + FateMutator putInitReserveColVal(FateId fateId); + + FateMutator requireReserved(FateId fateId); + FateMutator putName(byte[] data); FateMutator putAutoClean(byte[] data); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java index c794fdc3d62..28ad17144ab 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java @@ -49,6 +49,10 @@ public class FateMutatorImpl implements FateMutator { + // TODO 4131 these can be changed/simplified to take up less space/memory in the table + protected static final String IS_RESERVED = "isreserved"; + protected static final String NOT_RESERVED = "notreserved"; + private final ClientContext context; private final String tableName; private final FateId fateId; @@ -79,6 +83,44 @@ public FateMutator putCreateTime(long ctime) { return this; } + @Override + public FateMutator putReservedTx(FateId fateId) { + // Require that the column value is NOT_RESERVED (the FateId is not reserved) + Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()).setValue(NOT_RESERVED); + mutation.addCondition(condition); + TxColumnFamily.RESERVED_COLUMN.put(mutation, new Value(IS_RESERVED)); + return this; + } + + @Override + public FateMutator putUnreserveTx(FateId fateId) { + // Require that the column value is IS_RESERVED (the FateId is reserved) + Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()).setValue(IS_RESERVED); + mutation.addCondition(condition); + TxColumnFamily.RESERVED_COLUMN.put(mutation, new Value(NOT_RESERVED)); + return this; + } + + @Override + public FateMutator putInitReserveColVal(FateId fateId) { + // Require that the column does not have a set value yet + Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()); + mutation.addCondition(condition); + TxColumnFamily.RESERVED_COLUMN.put(mutation, new Value(NOT_RESERVED)); + return this; + } + + @Override + public FateMutator requireReserved(FateId fateId) { + Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()).setValue(IS_RESERVED); + mutation.addCondition(condition); + return this; + } + @Override public FateMutator putName(byte[] data) { TxInfoColumnFamily.TX_NAME_COLUMN.put(mutation, new Value(data)); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index c0e6623a93e..f0769b55e75 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -98,7 +98,7 @@ public FateId create() { } var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW) - .putCreateTime(System.currentTimeMillis()).tryMutate(); + .putCreateTime(System.currentTimeMillis()).putInitReserveColVal(fateId).tryMutate(); switch (status) { case ACCEPTED: @@ -131,7 +131,7 @@ protected void create(FateId fateId, FateKey fateKey) { } var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) - .putCreateTime(System.currentTimeMillis()).tryMutate(); + .putCreateTime(System.currentTimeMillis()).putInitReserveColVal(fateId).tryMutate(); switch (status) { case ACCEPTED: @@ -150,6 +150,39 @@ protected void create(FateId fateId, FateKey fateKey) { + " and fateKey " + fateKey + " after " + maxAttempts + " attempts"); } + @Override + public Optional> tryReserve(FateId fateId) { + // TODO 4131 should this throw an exception if the id doesn't exist (status = UNKNOWN)? + FateMutator.Status status = newMutator(fateId).putReservedTx(fateId).tryMutate(); + if (status.equals(FateMutator.Status.ACCEPTED)) { + return Optional.of(new FateTxStoreImpl(fateId, true)); + } else { + return Optional.empty(); + } + } + + @Override + protected boolean isReserved(FateId fateId) { + return newMutator(fateId).requireReserved(fateId).tryMutate() + .equals(FateMutator.Status.ACCEPTED); + } + + // TODO 4131 is public fine for this? Public for tests + @Override + public List getReservedTxns() { + try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { + scanner.setRange(new Range()); + scanner.fetchColumn(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()); + return scanner.stream() + .filter(e -> e.getValue().toString().equals(FateMutatorImpl.IS_RESERVED)) + .map(e -> FateId.from(fateInstanceType, e.getKey().getRow().toString())) + .collect(Collectors.toList()); + } catch (TableNotFoundException e) { + throw new IllegalStateException(tableName + " not found!", e); + } + } + @Override protected Stream getTransactions() { try { @@ -234,13 +267,8 @@ protected Pair> getStatusAndKey(FateId fateId) { } @Override - protected FateTxStore newFateTxStore(FateId fateId, boolean isReserved) { - return new FateTxStoreImpl(fateId, isReserved); - } - - @Override - protected FateInstanceType getInstanceType() { - return fateInstanceType; + protected FateTxStore newUnreservedFateTxStore(FateId fateId) { + return new FateTxStoreImpl(fateId, false); } static Range getRow(FateId fateId) { @@ -269,9 +297,16 @@ public FateInstanceType type() { } private class FateTxStoreImpl extends AbstractFateTxStoreImpl { + private boolean isReserved; private FateTxStoreImpl(FateId fateId, boolean isReserved) { - super(fateId, isReserved); + super(fateId); + this.isReserved = isReserved; + } + + @Override + protected boolean isReserved() { + return isReserved; } @Override @@ -401,6 +436,7 @@ public void delete() { var mutator = newMutator(fateId); mutator.requireStatus(TStatus.NEW, TStatus.SUBMITTED, TStatus.SUCCESSFUL, TStatus.FAILED); mutator.delete().mutate(); + this.deleted = true; } private Optional findTop() { @@ -411,6 +447,17 @@ private Optional findTop() { return scanner.stream().map(e -> restoreRepo(e.getKey().getColumnQualifier())).findFirst(); }); } + + @Override + protected void unreserve() { + if (!this.deleted) { + FateMutator.Status status = newMutator(fateId).putUnreserveTx(fateId).tryMutate(); + if (!status.equals(FateMutator.Status.ACCEPTED)) { + throw new IllegalStateException("Failed to unreserve " + fateId); + } + } + this.isReserved = false; + } } static Text invertRepo(int position) { diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java b/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java index b7b7846d189..07777687574 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java @@ -35,6 +35,9 @@ public static class TxColumnFamily { public static final String CREATE_TIME = "ctime"; public static final ColumnFQ CREATE_TIME_COLUMN = new ColumnFQ(NAME, new Text(CREATE_TIME)); + + public static final String RESERVED = "reserved"; + public static final ColumnFQ RESERVED_COLUMN = new ColumnFQ(NAME, new Text(RESERVED)); } public static class TxInfoColumnFamily { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index 09435c4d8fc..7dbf36ab522 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -62,6 +62,7 @@ import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.manager.thrift.TFateId; @@ -771,7 +772,9 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore mfs = new MetaFateStore<>(fateZkPath, zk); + // TODO 4131 dummy lock for now + MetaFateStore mfs = + new MetaFateStore<>(fateZkPath, zk, new ZooUtil.LockID("path", "node", 1234)); UserFateStore ufs = new UserFateStore<>(context); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index d17f5f570c3..866cb81c70d 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -1083,8 +1083,9 @@ boolean canSuspendTablets() { } try { - var metaInstance = initializeFateInstance(context, - new MetaFateStore<>(getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter())); + var metaInstance = + initializeFateInstance(context, new MetaFateStore<>(getZooKeeperRoot() + Constants.ZFATE, + context.getZooReaderWriter(), managerLock.getLockID())); var userInstance = initializeFateInstance(context, new UserFateStore<>(context, AccumuloTable.FATE.tableName())); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java index 15e8c4c44b1..3240aef0b6a 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java @@ -27,6 +27,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.metrics.MetricsUtil; import org.apache.accumulo.core.util.threads.ThreadPools; @@ -71,7 +72,9 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter()); + // TODO 4131 dummy lock for now + this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter(), + new ZooUtil.LockID("path", "node", 1234)); } catch (KeeperException ex) { throw new IllegalStateException( "FATE Metrics - Failed to create zoo store - metrics unavailable", ex); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java index 22fe50c0c90..a36266e6c9c 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java @@ -40,6 +40,7 @@ import org.apache.accumulo.core.conf.ConfigCheckUtil; import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.volume.Volume; @@ -307,8 +308,10 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - final ReadOnlyFateStore fate = new MetaFateStore<>( - context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()); + // TODO 4131 dummy lock for now + final ReadOnlyFateStore fate = + new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, + context.getZooReaderWriter(), new ZooUtil.LockID("path", "node", 1234)); try (var idStream = fate.list()) { if (idStream.findFirst().isPresent()) { throw new AccumuloException("Aborting upgrade because there are" diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index b77b2f59b66..3f1fe9c55a4 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -31,6 +31,7 @@ import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.row; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.verify; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.writeData; +import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.apache.accumulo.test.util.FileMetadataUtil.countFencedFiles; import static org.apache.accumulo.test.util.FileMetadataUtil.splitFilesIntoRanges; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -232,8 +233,8 @@ public void testExternalCompaction() throws Exception { @Test public void testCompactionCommitAndDeadDetectionRoot() throws Exception { var ctx = getCluster().getServerContext(); - FateStore metaFateStore = - new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter()); + FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, + ctx.getZooReaderWriter(), createTestLockID()); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var tableId = ctx.getTableId(AccumuloTable.ROOT.tableName()); @@ -251,8 +252,8 @@ public void testCompactionCommitAndDeadDetectionRoot() throws Exception { @Test public void testCompactionCommitAndDeadDetectionMeta() throws Exception { var ctx = getCluster().getServerContext(); - FateStore metaFateStore = - new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter()); + FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, + ctx.getZooReaderWriter(), createTestLockID()); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { // Metadata table by default already has 2 tablets @@ -298,8 +299,8 @@ public void testCompactionCommitAndDeadDetectionAll() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { UserFateStore userFateStore = new UserFateStore<>(ctx); - FateStore metaFateStore = - new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter()); + FateStore metaFateStore = new MetaFateStore<>( + ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), createTestLockID()); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java new file mode 100644 index 00000000000..f5f88fdf05a --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -0,0 +1,384 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.test.fate; + +import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.accumulo.core.client.Accumulo; +import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.conf.DefaultConfiguration; +import org.apache.accumulo.core.fate.AbstractFateStore; +import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FateStore; +import org.apache.accumulo.core.fate.MetaFateStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.fate.user.UserFateStore; +import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import com.google.common.collect.Sets; + +// TODO 4131 could potentially have separate classes for testing MetaFateStore and UserFateStore +public class MultipleStoresIT extends SharedMiniClusterBase { + + @TempDir + private static File tempDir; + private static ZooKeeperTestingServer szk = null; + private static ZooReaderWriter zk; + private static final String FATE_DIR = "/fate"; + private ClientContext client; + + @BeforeEach + public void beforeEachSetup() { + client = (ClientContext) Accumulo.newClient().from(getClientProps()).build(); + } + + @AfterEach + public void afterEachTeardown() { + client.close(); + } + + @BeforeAll + public static void beforeAllSetup() throws Exception { + SharedMiniClusterBase.startMiniCluster(); + szk = new ZooKeeperTestingServer(tempDir); + zk = szk.getZooReaderWriter(); + } + + @AfterAll + public static void afterAllTeardown() throws Exception { + SharedMiniClusterBase.stopMiniCluster(); + szk.close(); + } + + @Test + public void testReserveUnreserve() throws Exception { + testReserveUnreserve(FateInstanceType.META); + testReserveUnreserve(FateInstanceType.USER); + } + + protected void testReserveUnreserve(FateInstanceType storeType) throws Exception { + // reserving/unreserving a FateId should be reflected across instances of the stores + String tableName = getUniqueNames(1)[0]; + List expReservedList = new ArrayList<>(); + final int numFateIds = 500; + List> reservations = new ArrayList<>(); + boolean isUserStore = storeType.equals(FateInstanceType.USER); + Set allIds = new HashSet<>(); + final AbstractFateStore store1, store2; + + if (isUserStore) { + createFateTable(client, tableName); + } + + if (isUserStore) { + store1 = new UserFateStore<>(client, tableName); + store2 = new UserFateStore<>(client, tableName); + } else { + ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + store1 = new MetaFateStore<>(FATE_DIR, zk, lock1); + store2 = new MetaFateStore<>(FATE_DIR, zk, lock2); + } + + // Create the fate ids using store1 + for (int i = 0; i < numFateIds; i++) { + assertTrue(allIds.add(store1.create())); + } + assertEquals(numFateIds, allIds.size()); + + // Reserve half the fate ids using store1 and rest using store2, after reserving a fate id in + // one, should not be able to reserve the same in the other. Should also not matter that all the + // ids were created using store1 + int count = 0; + for (FateId fateId : allIds) { + expReservedList.add(fateId); + if (count % 2 == 0) { + reservations.add(store1.reserve(fateId)); + assertTrue(store2.tryReserve(fateId).isEmpty()); + } else { + reservations.add(store2.reserve(fateId)); + assertTrue(store1.tryReserve(fateId).isEmpty()); + } + count++; + } + // Both stores should return the same list of reserved transactions + assertTrue(expReservedList.containsAll(store1.getReservedTxns()) + && expReservedList.size() == store1.getReservedTxns().size()); + assertTrue(expReservedList.containsAll(store2.getReservedTxns()) + && expReservedList.size() == store2.getReservedTxns().size()); + + // Test setting/getting the TStatus and unreserving the transactions + for (int i = 0; i < allIds.size(); i++) { + var reservation = reservations.get(i); + assertEquals(ReadOnlyFateStore.TStatus.NEW, reservation.getStatus()); + reservation.setStatus(ReadOnlyFateStore.TStatus.SUBMITTED); + assertEquals(ReadOnlyFateStore.TStatus.SUBMITTED, reservation.getStatus()); + reservation.delete(); + reservation.unreserve(0, TimeUnit.MILLISECONDS); + // Attempt to set a status on a tx that has been unreserved (should throw exception) + try { + reservation.setStatus(ReadOnlyFateStore.TStatus.NEW); + fail(); + } catch (Exception e) { + // Expected + } + } + assertEquals(List.of(), store1.getReservedTxns()); + assertEquals(List.of(), store2.getReservedTxns()); + } + + @Test + public void testReserveReservedAndUnreserveUnreserved() throws Exception { + testReserveReservedAndUnreserveUnreserved(FateInstanceType.META); + testReserveReservedAndUnreserveUnreserved(FateInstanceType.USER); + } + + public void testReserveReservedAndUnreserveUnreserved(FateInstanceType storeType) + throws Exception { + String tableName = getUniqueNames(1)[0]; + final int numFateIds = 500; + boolean isUserStore = storeType.equals(FateInstanceType.USER); + Set allIds = new HashSet<>(); + List> reservations = new ArrayList<>(); + final AbstractFateStore store; + + if (isUserStore) { + createFateTable(client, tableName); + } + + if (isUserStore) { + store = new UserFateStore<>(client, tableName); + } else { + ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); + store = new MetaFateStore<>(FATE_DIR, zk, lock); + } + + // Create some FateIds and ensure that they can be reserved + for (int i = 0; i < numFateIds; i++) { + FateId fateId = store.create(); + assertTrue(allIds.add(fateId)); + var reservation = store.tryReserve(fateId); + assertFalse(reservation.isEmpty()); + reservations.add(reservation.orElseThrow()); + } + assertEquals(numFateIds, allIds.size()); + + // Try to reserve again, should not reserve + for (FateId fateId : allIds) { + assertTrue(store.tryReserve(fateId).isEmpty()); + } + + // Unreserve all the FateIds + for (var reservation : reservations) { + reservation.unreserve(0, TimeUnit.MILLISECONDS); + } + // Try to unreserve again (should throw exception) + for (var reservation : reservations) { + try { + reservation.unreserve(0, TimeUnit.MILLISECONDS); + fail(); + } catch (Exception e) { + // Expected + } + } + } + + @Test + public void testReserveAfterUnreserveAndReserveAfterDeleted() throws Exception { + testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType.META); + testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType.USER); + } + + public void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType storeType) + throws Exception { + String tableName = getUniqueNames(1)[0]; + final int numFateIds = 500; + boolean isUserStore = storeType.equals(FateInstanceType.USER); + Set allIds = new HashSet<>(); + List> reservations = new ArrayList<>(); + final AbstractFateStore store; + + if (isUserStore) { + createFateTable(client, tableName); + } + + if (isUserStore) { + store = new UserFateStore<>(client, tableName); + } else { + ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); + store = new MetaFateStore<>(FATE_DIR, zk, lock); + } + + // Create some FateIds and ensure that they can be reserved + for (int i = 0; i < numFateIds; i++) { + FateId fateId = store.create(); + assertTrue(allIds.add(fateId)); + var reservation = store.tryReserve(fateId); + assertFalse(reservation.isEmpty()); + reservations.add(reservation.orElseThrow()); + } + assertEquals(numFateIds, allIds.size()); + + // Unreserve all + for (var reservation : reservations) { + reservation.unreserve(0, TimeUnit.MILLISECONDS); + } + + // Ensure they can be reserved again, and delete and unreserve this time + for (FateId fateId : allIds) { + // Verify that the tx status is still NEW after unreserving since it hasn't been deleted + assertEquals(ReadOnlyFateStore.TStatus.NEW, store.read(fateId).getStatus()); + var reservation = store.tryReserve(fateId); + assertFalse(reservation.isEmpty()); + reservation.orElseThrow().delete(); + reservation.orElseThrow().unreserve(0, TimeUnit.MILLISECONDS); + } + + for (FateId fateId : allIds) { + // Verify that the tx is now unknown since it has been deleted + assertEquals(ReadOnlyFateStore.TStatus.UNKNOWN, store.read(fateId).getStatus()); + // Attempt to reserve a deleted txn, should throw an exception and not wait indefinitely + try { + store.reserve(fateId); + fail(); + } catch (Exception e) { + // Expected + } + } + } + + @Test + public void testMultipleFateInstances() throws Exception { + testMultipleFateInstances(FateInstanceType.META); + testMultipleFateInstances(FateInstanceType.USER); + } + + public void testMultipleFateInstances(FateInstanceType storeType) throws Exception { + String tableName = getUniqueNames(1)[0]; + final int numFateIds = 500; + boolean isUserStore = storeType.equals(FateInstanceType.USER); + Set allIds = new HashSet<>(); + final AbstractFateStore store1, store2; + final TestEnv testEnv1 = new TestEnv(); + final TestEnv testEnv2 = new TestEnv(); + + if (isUserStore) { + createFateTable(client, tableName); + } + + if (isUserStore) { + store1 = new UserFateStore<>(client, tableName); + store2 = new UserFateStore<>(client, tableName); + } else { + ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + store1 = new MetaFateStore<>(FATE_DIR, zk, lock1); + store2 = new MetaFateStore<>(FATE_DIR, zk, lock2); + } + + Fate fate1 = + new Fate<>(testEnv1, store1, Object::toString, DefaultConfiguration.getInstance()); + Fate fate2 = + new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); + + for (int i = 0; i < numFateIds; i++) { + FateId fateId; + // Start half the txns using fate1, and the other half using fate2 + if (i % 2 == 0) { + fateId = fate1.startTransaction(); + fate1.seedTransaction("op" + i, fateId, new TestRepo(), true, "test"); + } else { + fateId = fate2.startTransaction(); + fate2.seedTransaction("op" + i, fateId, new TestRepo(), true, "test"); + } + allIds.add(fateId); + } + assertEquals(numFateIds, allIds.size()); + + // Should be able to wait for completion on any fate instance + for (FateId fateId : allIds) { + fate2.waitForCompletion(fateId); + } + // Ensure that all txns have been executed and have only been executed once + assertTrue(Collections.disjoint(testEnv1.executedOps, testEnv2.executedOps)); + assertEquals(allIds, Sets.union(testEnv1.executedOps, testEnv2.executedOps)); + + fate1.shutdown(1, TimeUnit.MINUTES); + fate2.shutdown(1, TimeUnit.MINUTES); + } + + public static class TestRepo implements Repo { + private static final long serialVersionUID = 1L; + + @Override + public long isReady(FateId fateId, TestEnv environment) { + return 0; + } + + @Override + public String getName() { + return null; + } + + @Override + public Repo call(FateId fateId, TestEnv environment) throws Exception { + environment.executedOps.add(fateId); + Thread.sleep(50); // Simulate some work + return null; + } + + @Override + public void undo(FateId fateId, TestEnv environment) { + + } + + @Override + public String getReturn() { + return null; + } + } + + public static class TestEnv { + public final Set executedOps = Collections.synchronizedSet(new HashSet<>()); + } +} diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java index b9e8c101271..ddb30d253a5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java @@ -34,6 +34,7 @@ import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateIT; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; @@ -75,8 +76,12 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute( - new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred, fateIdGenerator), sctx); + testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, createTestLockID(), + maxDeferred, fateIdGenerator), sctx); + } + + public static ZooUtil.LockID createTestLockID() { + return new ZooUtil.LockID("S1", "N1", 1234); } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java index 5dc1daf9b29..70c9bdccc6b 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; + import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.MetaFateStore; @@ -32,6 +34,6 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, ServerContext sctx = getCluster().getServerContext(); String path = sctx.getZooKeeperRoot() + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); - testMethod.execute(new MetaFateStore<>(path, zk), sctx); + testMethod.execute(new MetaFateStore<>(path, zk, createTestLockID()), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java index b75b76a55be..c21c387275d 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java @@ -19,6 +19,7 @@ package org.apache.accumulo.test.fate.meta; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; +import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; @@ -75,8 +76,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - MetaFateStore store = - new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred, fateIdGenerator); + MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, + createTestLockID(), maxDeferred, fateIdGenerator); // Check that the store has no transactions before and after each test assertEquals(0, store.list().count()); @@ -89,28 +90,35 @@ protected void deleteKey(FateId fateId, ServerContext sctx) { try { // We have to use reflection since the NodeValue is internal to the store - // Grab both the constructor that uses the serialized bytes and status + // Grab both the constructors that use the serialized bytes and status, lock, uuid Class nodeClass = Class.forName(MetaFateStore.class.getName() + "$NodeValue"); - Constructor statusCons = nodeClass.getDeclaredConstructor(TStatus.class); + Constructor statusLockUUIDCons = + nodeClass.getDeclaredConstructor(TStatus.class, String.class, String.class); Constructor serializedCons = nodeClass.getDeclaredConstructor(byte[].class); - statusCons.setAccessible(true); + statusLockUUIDCons.setAccessible(true); serializedCons.setAccessible(true); - // Get the status field so it can be read and the serialize method + // Get the status, lock, and uuid fields so they can be read and the serialize method Field nodeStatus = nodeClass.getDeclaredField("status"); + Field nodeLock = nodeClass.getDeclaredField("lockID"); + Field nodeUUID = nodeClass.getDeclaredField("uuid"); Method nodeSerialize = nodeClass.getDeclaredMethod("serialize"); nodeStatus.setAccessible(true); + nodeLock.setAccessible(true); + nodeUUID.setAccessible(true); nodeSerialize.setAccessible(true); - // Get the existing status for the node and build a new node with an empty key + // Get the existing status, lock, and uuid for the node and build a new node with an empty key // but uses the existing tid String txPath = ZK_ROOT + Constants.ZFATE + "/tx_" + fateId.getTxUUIDStr(); Object currentNode = serializedCons.newInstance(new Object[] {zk.getData(txPath)}); TStatus currentStatus = (TStatus) nodeStatus.get(currentNode); - // replace the node with no key and just a tid and existing status - Object newNode = statusCons.newInstance(currentStatus); + String currentLock = (String) nodeLock.get(currentNode); + String currentUUID = (String) nodeUUID.get(currentNode); + // replace the node with no key and just a tid and existing status, lock, and uuid + Object newNode = statusLockUUIDCons.newInstance(currentStatus, currentLock, currentUUID); - // Replace the transaction with the same status and no key + // Replace the transaction with the same status, lock, and uuid and no key zk.putPersistentData(txPath, (byte[]) nodeSerialize.invoke(newNode), NodeExistsPolicy.OVERWRITE); } catch (Exception e) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java index 337f9e4bdc4..129d5632568 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java @@ -268,7 +268,7 @@ private void injectStatus(ClientContext client, String table, FateId fateId, TSt // Create the fate table with the exact configuration as the real Fate user instance table // including table properties and TabletAvailability - protected static void createFateTable(ClientContext client, String table) throws Exception { + public static void createFateTable(ClientContext client, String table) throws Exception { final var fateTableProps = client.tableOperations().getTableProperties(AccumuloTable.FATE.tableName()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index 5f05b0b64e8..210220b8cb1 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -21,6 +21,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -262,13 +263,13 @@ public void getFateStatus() { InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - MetaFateStore zs = - new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); + MetaFateStore mfs = new MetaFateStore<>( + ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createTestLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); UserFateStore ufs = new UserFateStore<>(context); Map> fateStores = - Map.of(FateInstanceType.META, zs, FateInstanceType.USER, ufs); + Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); withLocks = admin.getStatus(fateStores, zk, lockPath, null, null, null); @@ -356,11 +357,11 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - MetaFateStore zs = - new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); + MetaFateStore mfs = new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, + zk, createTestLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); - AdminUtil.FateStatus fateStatus = admin.getStatus(zs, zk, lockPath, null, null, null); + AdminUtil.FateStatus fateStatus = admin.getStatus(mfs, zk, lockPath, null, null, null); log.trace("current fates: {}", fateStatus.getTransactions().size()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java index 6d80cf5be06..825d12ed3d9 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java @@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FLUSH_ID; +import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -231,10 +232,10 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore zs = - new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk); + MetaFateStore mfs = + new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, createTestLockID()); var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); - return admin.getStatus(zs, zk, lockPath, null, null, null); + return admin.getStatus(mfs, zk, lockPath, null, null, null); } catch (KeeperException | InterruptedException e) { throw new RuntimeException(e); } From e6671fbb903da53014bd64e53ad125351add6a04 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Mon, 3 Jun 2024 10:11:26 -0400 Subject: [PATCH 02/17] Changes: - Combined the LockID and reservation attempt UUID into one new object: FateReservation - Now cover edge case with tryReserve() where a write may make it to the server, but the server dies before a response is received - New FATE Thread: DeadReservationCleaner which deletes reservations held by Managers that have since died - Created new classes: ColumnValueMappingIterator and ReservationMappingIterator. ColumnValueMappingIterator abstracts out the common functionality of ReservationMappingIterator and StatusMappingIterator. ReservationMappingIterator is an iterator used for determining if the reservation column for a FateId has a FateReservation set or not - Expanded/improved/simplified MultipleStoresIT tests --- .../accumulo/core/fate/AbstractFateStore.java | 54 ++- .../org/apache/accumulo/core/fate/Fate.java | 86 ++++- .../apache/accumulo/core/fate/FateStore.java | 162 ++++++++ .../accumulo/core/fate/MetaFateStore.java | 165 ++++---- .../fate/user/ColumnValueMappingIterator.java | 71 ++++ .../accumulo/core/fate/user/FateMutator.java | 58 ++- .../core/fate/user/FateMutatorImpl.java | 52 ++- .../fate/user/ReservationMappingIterator.java | 77 ++++ .../core/fate/user/StatusMappingIterator.java | 44 +-- .../core/fate/user/UserFateStore.java | 113 ++++-- .../core/fate/user/schema/FateSchema.java | 4 +- .../accumulo/core/logging/FateLogger.java | 21 + .../apache/accumulo/core/fate/TestStore.java | 22 ++ .../apache/accumulo/server/util/Admin.java | 7 +- .../org/apache/accumulo/manager/Manager.java | 5 +- .../manager/metrics/fate/FateMetrics.java | 6 +- .../manager/upgrade/UpgradeCoordinator.java | 5 +- .../compaction/ExternalCompaction_1_IT.java | 15 +- .../org/apache/accumulo/test/fate/FateIT.java | 4 +- .../accumulo/test/fate/FateOpsCommandsIT.java | 4 +- .../accumulo/test/fate/MultipleStoresIT.java | 363 +++++++++++++----- .../accumulo/test/fate/meta/MetaFateIT.java | 8 +- .../test/fate/meta/MetaFateOpsCommandsIT.java | 5 +- .../test/fate/meta/MetaFateStoreFateIT.java | 52 ++- .../test/fate/user/FateMutatorImplIT.java | 97 +++++ .../accumulo/test/fate/user/UserFateIT.java | 4 +- .../test/fate/user/UserFateOpsCommandsIT.java | 4 +- .../test/fate/user/UserFateStoreFateIT.java | 4 +- .../test/fate/user/UserFateStoreIT.java | 3 +- .../test/functional/FateConcurrencyIT.java | 15 +- .../test/functional/FunctionalTestUtils.java | 6 +- 31 files changed, 1187 insertions(+), 349 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 13b7f2c7ed8..8519937f203 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -31,7 +31,6 @@ import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -44,6 +43,9 @@ import java.util.stream.Stream; import org.apache.accumulo.core.fate.Fate.TxInfo; +import org.apache.accumulo.core.fate.zookeeper.ZooCache; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.time.NanoTime; @@ -54,6 +56,10 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +// TODO 4131 should probably add support to AbstractFateStore, MetaFateStore, +// and UserFateStore to accept null lockID and zooCache (maybe make these fields +// Optional<>). This could replace the current createDummyLockID(). This support +// is needed since MFS and UFS aren't always created in the context of a Manager. public abstract class AbstractFateStore implements FateStore { private static final Logger log = LoggerFactory.getLogger(AbstractFateStore.class); @@ -70,6 +76,9 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { } }; + // The ZooKeeper lock for the process that's running this store instance + protected final ZooUtil.LockID lockID; + protected final ZooCache zooCache; protected final Map deferred; private final int maxDeferred; private final AtomicBoolean deferredOverflow = new AtomicBoolean(); @@ -82,13 +91,20 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { private final AtomicInteger concurrentStatusChangeCallers = new AtomicInteger(0); public AbstractFateStore() { - this(DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + this(null, null, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } - public AbstractFateStore(int maxDeferred, FateIdGenerator fateIdGenerator) { + public AbstractFateStore(ZooUtil.LockID lockID, ZooCache zooCache) { + this(lockID, zooCache, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + } + + public AbstractFateStore(ZooUtil.LockID lockID, ZooCache zooCache, int maxDeferred, + FateIdGenerator fateIdGenerator) { this.maxDeferred = maxDeferred; this.fateIdGenerator = Objects.requireNonNull(fateIdGenerator); this.deferred = Collections.synchronizedMap(new HashMap<>()); + this.lockID = lockID; + this.zooCache = zooCache; } public static byte[] serialize(Object o) { @@ -317,6 +333,11 @@ public Optional> createAndReserve(FateKey fateKey) { return txStore; } + @Override + public boolean isDeadReservation(FateReservation reservation) { + return !ServiceLock.isLockHeld(zooCache, reservation.getLockID()); + } + protected abstract void create(FateId fateId, FateKey fateKey); protected abstract Pair> getStatusAndKey(FateId fateId); @@ -329,23 +350,28 @@ public Optional> createAndReserve(FateKey fateKey) { protected abstract FateTxStore newUnreservedFateTxStore(FateId fateId); - protected abstract boolean isReserved(FateId fateId); - - // TODO 4131 is public fine for this? Public for tests - public abstract List getReservedTxns(); - protected abstract class AbstractFateTxStoreImpl implements FateTxStore { protected final FateId fateId; protected boolean deleted; + protected FateReservation reservation; protected TStatus observedStatus = null; protected AbstractFateTxStoreImpl(FateId fateId) { this.fateId = fateId; this.deleted = false; + this.reservation = null; } - protected abstract boolean isReserved(); + protected AbstractFateTxStoreImpl(FateId fateId, FateReservation reservation) { + this.fateId = fateId; + this.deleted = false; + this.reservation = Objects.requireNonNull(reservation); + } + + protected boolean isReserved() { + return this.reservation != null; + } @Override public TStatus waitForStatusChange(EnumSet expected) { @@ -482,4 +508,14 @@ protected Serializable deserializeTxInfo(TxInfo txInfo, byte[] data) { throw new IllegalStateException("Bad node data " + txInfo); } } + + /** + * TODO 4131 this is a temporary method used to create a dummy lock when using a FateStore outside + * of the context of a Manager (one example is testing). + * + * @return a dummy {@link ZooUtil.LockID} + */ + public static ZooUtil.LockID createDummyLockID() { + return new ZooUtil.LockID("/path", "node", 123); + } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index eed785b39b2..b3d84b67c1f 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -74,10 +74,14 @@ public class Fate { private final ExecutorService executor; private static final EnumSet FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); + private static boolean userDeadReservationCleanerRunning = false; + private static boolean metaDeadReservationCleanerRunning = false; private final AtomicBoolean keepRunning = new AtomicBoolean(true); private final TransferQueue workQueue; private final Thread workFinder; + // Will be null if this Fate instance is not running a DeadReservationCleaner + private Thread deadReservationCleaner; public enum TxInfo { TX_NAME, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE @@ -289,6 +293,26 @@ private void undo(FateId fateId, Repo op) { } + /** + * A thread that finds reservations held by dead processes and unreserves them. Only one thread + * runs per store type across all Fate instances (one to clean up dead reservations for + * {@link org.apache.accumulo.core.fate.user.UserFateStore UserFateStore} and one to clean up dead + * reservations for {@link MetaFateStore}). + */ + private class DeadReservationCleaner implements Runnable { + // TODO 4131 periodic check runs every 30 seconds + // Should this be longer? Shorter? A configurable Property? A function of something? + private static final long INTERVAL_MILLIS = 30_000; + + @Override + public void run() { + while (keepRunning.get()) { + store.deleteDeadReservations(); + UtilWaitThread.sleep(INTERVAL_MILLIS); + } + } + } + /** * Creates a Fault-tolerant executor. * @@ -332,6 +356,33 @@ public Fate(T environment, FateStore store, Function,String> toLogStr this.workFinder.start(); } + /** + * Starts a thread that periodically cleans up "dead reservations" (see + * {@link FateStore#deleteDeadReservations()}). Only one thread is started per store type + * ({@link FateInstanceType}) for subsequent calls to this method. + */ + public void startDeadReservationCleaner() { + // TODO 4131 this is not ideal starting this thread in its own start method, but the other + // threads in the constructor. However, starting this thread in the constructor causes + // a Maven build failure, and do not want to move starting the other threads into a + // method in this PR... should be done standalone (see issue#4609). + + if ((store.type().equals(FateInstanceType.USER) && !userDeadReservationCleanerRunning) + || store.type().equals(FateInstanceType.META) && !metaDeadReservationCleanerRunning) { + if (store.type().equals(FateInstanceType.USER)) { + this.deadReservationCleaner = + Threads.createThread("USER dead reservation cleaner", new DeadReservationCleaner()); + userDeadReservationCleanerRunning = true; + } else if (store.type().equals(FateInstanceType.META)) { + this.deadReservationCleaner = + Threads.createThread("META dead reservation cleaner", new DeadReservationCleaner()); + metaDeadReservationCleanerRunning = true; + } + this.deadReservationCleaner.start(); + } + + } + // get a transaction id back to the requester before doing any work public FateId startTransaction() { return store.create(); @@ -498,13 +549,17 @@ public void shutdown(long timeout, TimeUnit timeUnit) { fatePoolWatcher.shutdown(); executor.shutdown(); workFinder.interrupt(); + if (deadReservationCleaner != null) { + deadReservationCleaner.interrupt(); + } } if (timeout > 0) { long start = System.nanoTime(); - while ((System.nanoTime() - start) < timeUnit.toNanos(timeout) - && (workFinder.isAlive() || !executor.isTerminated())) { + while ((System.nanoTime() - start) < timeUnit.toNanos(timeout) && (workFinder.isAlive() + || (deadReservationCleaner != null && deadReservationCleaner.isAlive()) + || !executor.isTerminated())) { try { if (!executor.awaitTermination(1, SECONDS)) { log.debug("Fate {} is waiting for worker threads to terminate", store.type()); @@ -516,19 +571,40 @@ public void shutdown(long timeout, TimeUnit timeUnit) { log.debug("Fate {} is waiting for work finder thread to terminate", store.type()); workFinder.interrupt(); } + + if (deadReservationCleaner != null) { + deadReservationCleaner.join(1_000); + } + if (deadReservationCleaner != null && deadReservationCleaner.isAlive()) { + log.debug("Fate {} is waiting for dead reservation cleaner thread to terminate", + store.type()); + deadReservationCleaner.interrupt(); + } } catch (InterruptedException e) { throw new RuntimeException(e); } } - if (workFinder.isAlive() || !executor.isTerminated()) { + if (workFinder.isAlive() + || (deadReservationCleaner != null && deadReservationCleaner.isAlive()) + || !executor.isTerminated()) { log.warn( - "Waited for {}ms for all fate {} background threads to stop, but some are still running. workFinder:{} executor:{}", + "Waited for {}ms for all fate {} background threads to stop, but some are still running. workFinder:{} deadReservationCleaner:{} executor:{}", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), store.type(), - workFinder.isAlive(), !executor.isTerminated()); + workFinder.isAlive(), + (deadReservationCleaner != null && deadReservationCleaner.isAlive()), + !executor.isTerminated()); } } + // Update that USER/META dead reservation cleaner is no longer running + if (deadReservationCleaner != null && !deadReservationCleaner.isAlive()) { + if (store.type().equals(FateInstanceType.USER)) { + userDeadReservationCleanerRunning = false; + } else if (store.type().equals(FateInstanceType.META)) { + metaDeadReservationCleanerRunning = false; + } + } // interrupt the background threads executor.shutdownNow(); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index f93113112a3..1161f4ca2c4 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -18,9 +18,20 @@ */ package org.apache.accumulo.core.fate; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Map; +import java.util.Objects; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; + +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.hadoop.io.DataInputBuffer; /** * Transaction Store: a place to save transactions @@ -107,6 +118,157 @@ interface FateTxStore extends ReadOnlyFateTxStore { void unreserve(long deferTime, TimeUnit timeUnit); } + /** + * The value stored to indicate a FATE transaction ID ({@link FateId}) has been reserved + */ + class FateReservation { + + // The LockID (provided by the Manager running the FATE which uses this store) which is used for + // identifying dead Managers, so their reservations can be deleted and picked up again since + // they can no longer be worked on. + private final ZooUtil.LockID lockID; // TODO 4131 not sure if this is the best type for this + // The UUID generated on a reservation attempt (tryReserve()) used to uniquely identify that + // attempt. This is useful for the edge case where the reservation is sent to the server + // (Tablet Server for UserFateStore and the ZooKeeper Server for MetaFateStore), but the server + // dies before the store receives the response. It allows us to determine if the reservation + // was successful and was written by this reservation attempt (could have been successfully + // reserved by another attempt or not reserved at all, in which case, we wouldn't want to + // expose a FateTxStore). + private final UUID reservationUUID; + private final byte[] serialized; + private static final Pattern UUID_PATTERN = + Pattern.compile("^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"); + private static final Pattern LOCKID_PATTERN = Pattern.compile("^.+/.+\\$[0-9a-fA-F]+$"); + + private FateReservation(ZooUtil.LockID lockID, UUID reservationUUID) { + this.lockID = Objects.requireNonNull(lockID); + this.reservationUUID = Objects.requireNonNull(reservationUUID); + this.serialized = serialize(lockID, reservationUUID); + } + + public static FateReservation from(ZooUtil.LockID lockID, UUID reservationUUID) { + return new FateReservation(lockID, reservationUUID); + } + + public static FateReservation from(byte[] serialized) { + try (DataInputBuffer buffer = new DataInputBuffer()) { + buffer.reset(serialized, serialized.length); + ZooUtil.LockID lockID = new ZooUtil.LockID("", buffer.readUTF()); + UUID reservationUUID = UUID.fromString(buffer.readUTF()); + return new FateReservation(lockID, reservationUUID); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static FateReservation from(String fateReservationStr) { + if (isFateReservation(fateReservationStr)) { + String[] fields = fateReservationStr.split(":"); + ZooUtil.LockID lockId = new ZooUtil.LockID("", fields[0]); + UUID reservationUUID = UUID.fromString(fields[1]); + return new FateReservation(lockId, reservationUUID); + } else { + throw new IllegalArgumentException( + "Tried to create a FateReservation from an invalid string: " + fateReservationStr); + } + } + + /** + * + * @param fateReservationStr the string from a call to FateReservations toString() + * @return true if the string represents a valid FateReservation object, false otherwise + */ + public static boolean isFateReservation(String fateReservationStr) { + if (fateReservationStr != null) { + String[] fields = fateReservationStr.split(":"); + if (fields.length == 2) { + return LOCKID_PATTERN.matcher(fields[0]).matches() + && UUID_PATTERN.matcher(fields[1]).matches(); + } + } + return false; + } + + public ZooUtil.LockID getLockID() { + return lockID; + } + + public UUID getReservationUUID() { + return reservationUUID; + } + + public byte[] getSerialized() { + return serialized; + } + + private static byte[] serialize(ZooUtil.LockID lockID, UUID reservationUUID) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos)) { + dos.writeUTF(lockID.serialize("/")); + dos.writeUTF(reservationUUID.toString()); + dos.close(); + return baos.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static FateReservation deserialize(byte[] serialized) { + return FateReservation.from(serialized); + } + + public static boolean locksAreEqual(ZooUtil.LockID lockID1, ZooUtil.LockID lockID2) { + return lockID1.serialize("/").equals(lockID2.serialize("/")); + } + + @Override + public String toString() { + return lockID.serialize("/") + ":" + reservationUUID; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof FateReservation) { + FateReservation other = (FateReservation) obj; + return this.lockID.serialize("/").equals(other.lockID.serialize("/")) + && this.reservationUUID.equals(other.reservationUUID); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(lockID, reservationUUID); + } + } + + /** + * @param fateId the fateId to check + * @return true if the given fate id is reserved, false otherwise + */ + boolean isReserved(FateId fateId); + + /** + * @return a map of the current active reservations with the keys being the transaction that is + * reserved and the value being the value stored to indicate the transaction is reserved. + */ + Map getActiveReservations(); + + /** + * Deletes the current reservations which were reserved by a now dead Manager. These reservations + * can no longer be worked on so their reservation should be deleted, so they can be picked up and + * worked on again. + */ + void deleteDeadReservations(); + + /** + * The way dead reservations are determined for {@link #deleteDeadReservations()} + * + * @param reservation the fate reservation + * @return true if reservation held by a dead Manager, false otherwise + */ + boolean isDeadReservation(FateReservation reservation); + /** * Attempt to reserve the fate transaction. * diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index 612a60d6fc8..d6f34d343c4 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -28,16 +28,18 @@ import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.UUID; import java.util.function.Supplier; -import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.fate.Fate.TxInfo; +import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; @@ -63,25 +65,23 @@ public class MetaFateStore extends AbstractFateStore { private static final FateInstanceType fateInstanceType = FateInstanceType.META; private String path; private ZooReaderWriter zk; - // The ZooKeeper lock for the process that's running this store instance - private ZooUtil.LockID lockID; private String getTXPath(FateId fateId) { return path + "/tx_" + fateId.getTxUUIDStr(); } - public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID) + public MetaFateStore(String path, ZooReaderWriter zk, ZooCache zooCache, ZooUtil.LockID lockID) throws KeeperException, InterruptedException { - this(path, zk, lockID, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + this(path, zk, zooCache, lockID, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } @VisibleForTesting - public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID, int maxDeferred, - FateIdGenerator fateIdGenerator) throws KeeperException, InterruptedException { - super(maxDeferred, fateIdGenerator); + public MetaFateStore(String path, ZooReaderWriter zk, ZooCache zooCache, ZooUtil.LockID lockID, + int maxDeferred, FateIdGenerator fateIdGenerator) + throws KeeperException, InterruptedException { + super(lockID, zooCache, maxDeferred, fateIdGenerator); this.path = path; this.zk = zk; - this.lockID = lockID; zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); } @@ -96,7 +96,7 @@ public FateId create() { while (true) { try { FateId fateId = FateId.from(fateInstanceType, UUID.randomUUID()); - zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, "", "").serialize(), + zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, null).serialize(), NodeExistsPolicy.FAIL); return fateId; } catch (NodeExistsException nee) { @@ -110,7 +110,7 @@ public FateId create() { @Override protected void create(FateId fateId, FateKey key) { try { - zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, "", "", key).serialize(), + zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, null, key).serialize(), NodeExistsPolicy.FAIL); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); @@ -119,8 +119,12 @@ protected void create(FateId fateId, FateKey key) { @Override public Optional> tryReserve(FateId fateId) { + // return an empty option if the FateId doesn't exist + if (_getStatus(fateId).equals(TStatus.UNKNOWN)) { + return Optional.empty(); + } // uniquely identify this attempt to reserve the fate operation data - UUID uuid = UUID.randomUUID(); + FateReservation reservation = FateReservation.from(lockID, UUID.randomUUID()); try { byte[] newSerNodeVal = zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { @@ -128,18 +132,18 @@ public Optional> tryReserve(FateId fateId) { // The uuid handles the case where there was a ZK server fault and the write for this thread // went through but that was not acknowledged, and we are reading our own write for 2nd // time. - if (!currNodeVal.isReserved() || currNodeVal.uuid.equals(uuid.toString())) { + if (!currNodeVal.isReserved() || (currNodeVal.isReserved() + && currNodeVal.reservation.orElseThrow().equals(reservation))) { FateKey currFateKey = currNodeVal.fateKey.orElse(null); - // Add the lock and uuid to the node to reserve - return new NodeValue(currNodeVal.status, lockID.serialize(""), uuid.toString(), - currFateKey).serialize(); + // Add the FateReservation to the node to reserve + return new NodeValue(currNodeVal.status, reservation, currFateKey).serialize(); } else { // This will not change the value to null but will return null return null; } }); if (newSerNodeVal != null) { - return Optional.of(new FateTxStoreImpl(fateId, uuid)); + return Optional.of(new FateTxStoreImpl(fateId, reservation)); } else { return Optional.empty(); } @@ -149,7 +153,7 @@ public Optional> tryReserve(FateId fateId) { } @Override - protected boolean isReserved(FateId fateId) { + public boolean isReserved(FateId fateId) { boolean isReserved; try { isReserved = getNode(fateId).isReserved(); @@ -160,20 +164,50 @@ protected boolean isReserved(FateId fateId) { return isReserved; } - // TODO 4131 is public fine for this? Public for tests @Override - public List getReservedTxns() { + public Map getActiveReservations() { + Map activeReservations = new HashMap<>(); + try { - return zk.getChildren(path).stream().filter(strTxId -> { - String txUUIDStr = strTxId.split("_")[1]; - return isReserved(FateId.from(fateInstanceType, txUUIDStr)); - }).map(strTxId -> { + for (String strTxId : zk.getChildren(path)) { String txUUIDStr = strTxId.split("_")[1]; - return FateId.from(fateInstanceType, txUUIDStr); - }).collect(Collectors.toList()); + FateId fateId = FateId.from(fateInstanceType, txUUIDStr); + if (isReserved(fateId)) { + FateReservation reservation = getNode(fateId).reservation.orElseThrow(); + activeReservations.put(fateId, reservation); + } + } } catch (KeeperException | InterruptedException e) { throw new RuntimeException(e); } + + return activeReservations; + } + + @Override + public void deleteDeadReservations() { + for (Map.Entry entry : getActiveReservations().entrySet()) { + FateId fateId = entry.getKey(); + FateReservation reservation = entry.getValue(); + try { + zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { + NodeValue currNodeVal = new NodeValue(currSerNodeVal); + // Make sure the current node is still reserved and reserved with the expected reservation + // and it is dead + if (currNodeVal.isReserved() && currNodeVal.reservation.orElseThrow().equals(reservation) + && isDeadReservation(currNodeVal.reservation.orElseThrow())) { + // Delete the reservation + return new NodeValue(currNodeVal.status, null, currNodeVal.fateKey.orElse(null)) + .serialize(); + } else { + // No change + return null; + } + }); + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { + throw new RuntimeException(e); + } + } } @Override @@ -188,21 +222,13 @@ public FateInstanceType type() { } private class FateTxStoreImpl extends AbstractFateTxStoreImpl { - private UUID reservationUUID; private FateTxStoreImpl(FateId fateId) { super(fateId); - this.reservationUUID = null; } - private FateTxStoreImpl(FateId fateId, UUID reservationUUID) { - super(fateId); - this.reservationUUID = Objects.requireNonNull(reservationUUID); - } - - @Override - protected boolean isReserved() { - return reservationUUID != null; + private FateTxStoreImpl(FateId fateId, FateReservation reservation) { + super(fateId, reservation); } private static final int RETRIES = 10; @@ -301,9 +327,9 @@ public void setStatus(TStatus status) { try { zk.mutateExisting(getTXPath(fateId), currSerializedData -> { NodeValue currNodeVal = new NodeValue(currSerializedData); + FateReservation currFateReservation = currNodeVal.reservation.orElse(null); FateKey currFateKey = currNodeVal.fateKey.orElse(null); - NodeValue newNodeValue = - new NodeValue(status, currNodeVal.lockID, currNodeVal.uuid, currFateKey); + NodeValue newNodeValue = new NodeValue(status, currFateReservation, currFateKey); return newNodeValue.serialize(); }); } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { @@ -404,9 +430,10 @@ protected void unreserve() { zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { NodeValue currNodeVal = new NodeValue(currSerNodeVal); FateKey currFateKey = currNodeVal.fateKey.orElse(null); - if (currNodeVal.uuid.equals(reservationUUID.toString())) { - // Remove the lock and uuid from the NodeValue to unreserve - return new NodeValue(currNodeVal.status, "", "", currFateKey).serialize(); + if ((currNodeVal.isReserved() + && currNodeVal.reservation.orElseThrow().equals(this.reservation))) { + // Remove the FateReservation from the NodeValue to unreserve + return new NodeValue(currNodeVal.status, null, currFateKey).serialize(); } else { // possible this is running a 2nd time in zk server fault conditions and its first // write went through @@ -414,7 +441,7 @@ protected void unreserve() { } }); } - this.reservationUUID = null; + this.reservation = null; } catch (InterruptedException | KeeperException | AcceptableThriftTableOperationException e) { throw new IllegalStateException(e); } @@ -445,7 +472,7 @@ private NodeValue getNode(FateId fateId) { try { return new NodeValue(zk.getData(getTXPath(fateId))); } catch (NoNodeException nne) { - return new NodeValue(TStatus.UNKNOWN, "", ""); + return new NodeValue(TStatus.UNKNOWN, null); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -486,34 +513,26 @@ public Stream list(FateKey.FateKeyType type) { protected static class NodeValue { final TStatus status; final Optional fateKey; - final String lockID; - final String uuid; + final Optional reservation; private NodeValue(byte[] serializedData) { try (DataInputBuffer buffer = new DataInputBuffer()) { buffer.reset(serializedData, serializedData.length); - TStatus tempStatus = TStatus.valueOf(buffer.readUTF()); - String tempLockID = buffer.readUTF(); - String tempUUID = buffer.readUTF(); - validateLockAndUUID(tempLockID, tempUUID); - this.status = tempStatus; - this.lockID = tempLockID; - this.uuid = tempUUID; + this.status = TStatus.valueOf(buffer.readUTF()); + this.reservation = deserializeFateReservation(buffer); this.fateKey = deserializeFateKey(buffer); } catch (IOException e) { throw new UncheckedIOException(e); } } - private NodeValue(TStatus status, String lockID, String uuid) { - this(status, lockID, uuid, null); + private NodeValue(TStatus status, FateReservation reservation) { + this(status, reservation, null); } - private NodeValue(TStatus status, String lockID, String uuid, FateKey fateKey) { - validateLockAndUUID(lockID, uuid); - this.status = status; - this.lockID = lockID; - this.uuid = uuid; + private NodeValue(TStatus status, FateReservation reservation, FateKey fateKey) { + this.status = Objects.requireNonNull(status); + this.reservation = Optional.ofNullable(reservation); this.fateKey = Optional.ofNullable(fateKey); } @@ -525,12 +544,26 @@ private Optional deserializeFateKey(DataInputBuffer buffer) throws IOEx return Optional.empty(); } + private Optional deserializeFateReservation(DataInputBuffer buffer) + throws IOException { + int length = buffer.readInt(); + if (length > 0) { + return Optional.of(FateReservation.deserialize(buffer.readNBytes(length))); + } + return Optional.empty(); + } + byte[] serialize() { try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos)) { dos.writeUTF(status.name()); - dos.writeUTF(lockID); - dos.writeUTF(uuid); + if (isReserved()) { + byte[] serializedFateReservation = reservation.orElseThrow().getSerialized(); + dos.writeInt(serializedFateReservation.length); + dos.write(serializedFateReservation); + } else { + dos.writeInt(0); + } if (fateKey.isPresent()) { byte[] serializedFateKey = fateKey.orElseThrow().getSerialized(); dos.writeInt(serializedFateKey.length); @@ -546,15 +579,7 @@ byte[] serialize() { } public boolean isReserved() { - return !lockID.isEmpty(); - } - - private void validateLockAndUUID(String lockID, String uuid) { - // TODO 4131 potentially need further validation? - if (!((lockID.isEmpty() && uuid.isEmpty()) || (!lockID.isEmpty() && !uuid.isEmpty()))) { - throw new IllegalArgumentException( - "One but not both of lock = '" + lockID + "' and uuid = '" + uuid + "' are empty"); - } + return reservation.isPresent(); } } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java new file mode 100644 index 00000000000..1d73608350a --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.fate.user; + +import java.io.IOException; +import java.util.Collection; +import java.util.Objects; + +import org.apache.accumulo.core.data.ByteSequence; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.IteratorEnvironment; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; + +public abstract class ColumnValueMappingIterator implements SortedKeyValueIterator { + + protected SortedKeyValueIterator source; + protected Value mappedValue; + + protected abstract void mapValue(); + + @Override + public boolean hasTop() { + return source.hasTop(); + } + + @Override + public void next() throws IOException { + source.next(); + mapValue(); + } + + @Override + public void seek(Range range, Collection columnFamilies, boolean inclusive) + throws IOException { + source.seek(range, columnFamilies, inclusive); + mapValue(); + } + + @Override + public Key getTopKey() { + return source.getTopKey(); + } + + @Override + public Value getTopValue() { + return Objects.requireNonNull(mappedValue); + } + + @Override + public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { + throw new UnsupportedOperationException(); + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java index 8042df117cb..e06d8638eac 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java @@ -19,10 +19,11 @@ package org.apache.accumulo.core.fate.user; import org.apache.accumulo.core.fate.Fate; -import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateKey; +import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.fate.user.schema.FateSchema; public interface FateMutator { @@ -32,13 +33,54 @@ public interface FateMutator { FateMutator putCreateTime(long ctime); - FateMutator putReservedTx(FateId fateId); - - FateMutator putUnreserveTx(FateId fateId); - - FateMutator putInitReserveColVal(FateId fateId); - - FateMutator requireReserved(FateId fateId); + /** + * Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will + * put the reservation if there is not already a reservation present + * + * @param reservation the reservation to attempt to put + * @return the FateMutator with this added mutation + */ + FateMutator putReservedTx(FateStore.FateReservation reservation); + + /** + * Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will + * remove the given reservation if it matches what is present in the column. + * + * @param reservation the reservation to attempt to remove + * @return the FateMutator with this added mutation + */ + FateMutator putUnreserveTx(FateStore.FateReservation reservation); + + /** + * Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will + * put the initial column value if it has not already been set yet + * + * @return the FateMutator with this added mutation + */ + FateMutator putInitReserveColVal(); + + /** + * Require that the transaction is reserved with a specific {@link FateStore.FateReservation} + * + * @param reservation the reservation + * @return the FateMutator with the added condition + */ + FateMutator requireReserved(FateStore.FateReservation reservation); + + /** + * Require that the transaction is reserved (can be reserved with any + * {@link FateStore.FateReservation}) + * + * @return the FateMutator with the added condition + */ + FateMutator requireReserved(); + + /** + * Require that the transaction is unreserved + * + * @return the FateMutator with the added condition + */ + FateMutator requireUnreserved(); FateMutator putName(byte[] data); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java index 28ad17144ab..550e21755ab 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java @@ -39,6 +39,7 @@ import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateKey; +import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.user.schema.FateSchema.RepoColumnFamily; @@ -49,9 +50,7 @@ public class FateMutatorImpl implements FateMutator { - // TODO 4131 these can be changed/simplified to take up less space/memory in the table - protected static final String IS_RESERVED = "isreserved"; - protected static final String NOT_RESERVED = "notreserved"; + protected static final String NOT_RESERVED = ""; private final ClientContext context; private final String tableName; @@ -84,39 +83,50 @@ public FateMutator putCreateTime(long ctime) { } @Override - public FateMutator putReservedTx(FateId fateId) { - // Require that the column value is NOT_RESERVED (the FateId is not reserved) - Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()).setValue(NOT_RESERVED); + public FateMutator putReservedTx(FateStore.FateReservation reservation) { + Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()).setValue(NOT_RESERVED); mutation.addCondition(condition); - TxColumnFamily.RESERVED_COLUMN.put(mutation, new Value(IS_RESERVED)); + TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(reservation.toString())); return this; } @Override - public FateMutator putUnreserveTx(FateId fateId) { - // Require that the column value is IS_RESERVED (the FateId is reserved) - Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()).setValue(IS_RESERVED); + public FateMutator putUnreserveTx(FateStore.FateReservation reservation) { + Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()).setValue(reservation.toString()); mutation.addCondition(condition); - TxColumnFamily.RESERVED_COLUMN.put(mutation, new Value(NOT_RESERVED)); + TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(NOT_RESERVED)); return this; } @Override - public FateMutator putInitReserveColVal(FateId fateId) { - // Require that the column does not have a set value yet - Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()); + public FateMutator putInitReserveColVal() { + Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); mutation.addCondition(condition); - TxColumnFamily.RESERVED_COLUMN.put(mutation, new Value(NOT_RESERVED)); + TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(NOT_RESERVED)); return this; } @Override - public FateMutator requireReserved(FateId fateId) { - Condition condition = new Condition(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()).setValue(IS_RESERVED); + public FateMutator requireReserved(FateStore.FateReservation reservation) { + Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()).setValue(reservation.toString()); + mutation.addCondition(condition); + return this; + } + + @Override + public FateMutator requireReserved() { + Condition condition = ReservationMappingIterator.createRequireReservedCondition(); + mutation.addCondition(condition); + return this; + } + + @Override + public FateMutator requireUnreserved() { + Condition condition = ReservationMappingIterator.createRequireUnreservedCondition(); mutation.addCondition(condition); return this; } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java new file mode 100644 index 00000000000..8a04c435f14 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.fate.user; + +import static org.apache.accumulo.core.fate.user.schema.FateSchema.TxColumnFamily.RESERVATION_COLUMN; + +import java.io.IOException; +import java.util.Map; + +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.data.Condition; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.fate.FateStore; +import org.apache.accumulo.core.iterators.IteratorEnvironment; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; + +/** + * An iterator used for determining if the reservation column for a FateId has a FateReservation set + * or not. Maps the value of the column to "isreserved" or "notreserved" if the column has a + * FateReservation value set or not. + */ +public class ReservationMappingIterator extends ColumnValueMappingIterator { + + private static final String IS_RESERVED = "isreserved"; + private static final String NOT_RESERVED = "notreserved"; + + @Override + public void init(SortedKeyValueIterator source, Map options, + IteratorEnvironment env) throws IOException { + this.source = source; + // No need for options or env + } + + @Override + protected void mapValue() { + if (hasTop()) { + String currVal = source.getTopValue().toString(); + mappedValue = FateStore.FateReservation.isFateReservation(currVal) ? new Value(IS_RESERVED) + : new Value(NOT_RESERVED); + } else { + mappedValue = null; + } + } + + public static Condition createRequireReservedCondition() { + Condition condition = new Condition(RESERVATION_COLUMN.getColumnFamily(), + RESERVATION_COLUMN.getColumnQualifier()); + IteratorSetting is = new IteratorSetting(100, ReservationMappingIterator.class); + + return condition.setValue(IS_RESERVED).setIterators(is); + } + + public static Condition createRequireUnreservedCondition() { + Condition condition = new Condition(RESERVATION_COLUMN.getColumnFamily(), + RESERVATION_COLUMN.getColumnQualifier()); + IteratorSetting is = new IteratorSetting(100, ReservationMappingIterator.class); + + return condition.setValue(NOT_RESERVED).setIterators(is); + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java index 1a0fae5aa3e..d9c667b25b4 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java @@ -22,18 +22,14 @@ import java.io.IOException; import java.util.Arrays; -import java.util.Collection; import java.util.HashSet; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Condition; import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.iterators.IteratorEnvironment; @@ -44,15 +40,13 @@ * iterator allows for checking of the status column's value against a set of acceptable statuses * within a conditional mutation. */ -public class StatusMappingIterator implements SortedKeyValueIterator { +public class StatusMappingIterator extends ColumnValueMappingIterator { private static final String PRESENT = "present"; private static final String ABSENT = "absent"; private static final String STATUS_SET_KEY = "statusSet"; - private SortedKeyValueIterator source; private final Set acceptableStatuses = new HashSet<>(); - private Value mappedValue; /** * The set of acceptable must be provided as an option to the iterator using the @@ -70,29 +64,12 @@ public void init(SortedKeyValueIterator source, Map op } } - @Override - public boolean hasTop() { - return source.hasTop(); - } - - @Override - public void next() throws IOException { - source.next(); - mapValue(); - } - - @Override - public void seek(Range range, Collection columnFamilies, boolean inclusive) - throws IOException { - source.seek(range, columnFamilies, inclusive); - mapValue(); - } - /** * Maps the value of the status column to "present" or "absent" based on its presence within the * set of statuses. */ - private void mapValue() { + @Override + protected void mapValue() { if (source.hasTop()) { String currentValue = source.getTopValue().toString(); mappedValue = @@ -102,21 +79,6 @@ private void mapValue() { } } - @Override - public Key getTopKey() { - return source.getTopKey(); - } - - @Override - public Value getTopValue() { - return Objects.requireNonNull(mappedValue); - } - - @Override - public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { - throw new UnsupportedOperationException(); - } - /** * Creates a condition that checks if the status column's value is one of the given acceptable * statuses. diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index f0769b55e75..98385a5ffc1 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -19,7 +19,9 @@ package org.apache.accumulo.core.fate.user; import java.io.Serializable; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; @@ -45,6 +47,7 @@ import org.apache.accumulo.core.fate.user.schema.FateSchema.RepoColumnFamily; import org.apache.accumulo.core.fate.user.schema.FateSchema.TxColumnFamily; import org.apache.accumulo.core.fate.user.schema.FateSchema.TxInfoColumnFamily; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.ColumnFQ; @@ -69,20 +72,20 @@ public class UserFateStore extends AbstractFateStore { private static final com.google.common.collect.Range REPO_RANGE = com.google.common.collect.Range.closed(1, maxRepos); - public UserFateStore(ClientContext context, String tableName) { - this(context, tableName, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + public UserFateStore(ClientContext context, String tableName, ZooUtil.LockID lockID) { + this(context, tableName, lockID, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } @VisibleForTesting - public UserFateStore(ClientContext context, String tableName, int maxDeferred, - FateIdGenerator fateIdGenerator) { - super(maxDeferred, fateIdGenerator); + public UserFateStore(ClientContext context, String tableName, ZooUtil.LockID lockID, + int maxDeferred, FateIdGenerator fateIdGenerator) { + super(lockID, context.getZooCache(), maxDeferred, fateIdGenerator); this.context = Objects.requireNonNull(context); this.tableName = Objects.requireNonNull(tableName); } - public UserFateStore(ClientContext context) { - this(context, AccumuloTable.FATE.tableName()); + public UserFateStore(ClientContext context, ZooUtil.LockID lockID) { + this(context, AccumuloTable.FATE.tableName(), lockID); } @Override @@ -98,7 +101,7 @@ public FateId create() { } var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW) - .putCreateTime(System.currentTimeMillis()).putInitReserveColVal(fateId).tryMutate(); + .putCreateTime(System.currentTimeMillis()).putInitReserveColVal().tryMutate(); switch (status) { case ACCEPTED: @@ -131,7 +134,7 @@ protected void create(FateId fateId, FateKey fateKey) { } var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) - .putCreateTime(System.currentTimeMillis()).putInitReserveColVal(fateId).tryMutate(); + .putCreateTime(System.currentTimeMillis()).putInitReserveColVal().tryMutate(); switch (status) { case ACCEPTED: @@ -152,35 +155,72 @@ protected void create(FateId fateId, FateKey fateKey) { @Override public Optional> tryReserve(FateId fateId) { - // TODO 4131 should this throw an exception if the id doesn't exist (status = UNKNOWN)? - FateMutator.Status status = newMutator(fateId).putReservedTx(fateId).tryMutate(); + // Create a unique FateReservation for this reservation attempt + FateReservation reservation = FateReservation.from(lockID, UUID.randomUUID()); + + FateMutator.Status status = newMutator(fateId).putReservedTx(reservation).tryMutate(); if (status.equals(FateMutator.Status.ACCEPTED)) { - return Optional.of(new FateTxStoreImpl(fateId, true)); - } else { - return Optional.empty(); + return Optional.of(new FateTxStoreImpl(fateId, reservation)); + } else if (status.equals(FateMutator.Status.UNKNOWN)) { + // If the status is UNKNOWN, this means an error occurred after the mutation was + // sent to the TabletServer, and it is unknown if the mutation was written. We + // need to check if the mutation was written and if it was written by this + // attempt at reservation. If it was written by this reservation attempt, + // we can return the FateTxStore since it was successfully reserved in this + // attempt, otherwise we return empty (was written by another reservation + // attempt or was not written at all). + status = newMutator(fateId).requireReserved(reservation).tryMutate(); + if (status.equals(FateMutator.Status.ACCEPTED)) { + return Optional.of(new FateTxStoreImpl(fateId, reservation)); + } } + return Optional.empty(); } @Override - protected boolean isReserved(FateId fateId) { - return newMutator(fateId).requireReserved(fateId).tryMutate() - .equals(FateMutator.Status.ACCEPTED); + public boolean isReserved(FateId fateId) { + return newMutator(fateId).requireReserved().tryMutate().equals(FateMutator.Status.ACCEPTED); } - // TODO 4131 is public fine for this? Public for tests @Override - public List getReservedTxns() { + public Map getActiveReservations() { + Map activeReservations = new HashMap<>(); + try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { scanner.setRange(new Range()); - scanner.fetchColumn(TxColumnFamily.RESERVED_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVED_COLUMN.getColumnQualifier()); - return scanner.stream() - .filter(e -> e.getValue().toString().equals(FateMutatorImpl.IS_RESERVED)) - .map(e -> FateId.from(fateInstanceType, e.getKey().getRow().toString())) - .collect(Collectors.toList()); + scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); + scanner.stream() + .filter(entry -> FateReservation.isFateReservation(entry.getValue().toString())) + .forEach(entry -> { + String reservationColVal = entry.getValue().toString(); + FateId fateId = FateId.from(fateInstanceType, entry.getKey().getRow().toString()); + FateReservation reservation = FateReservation.from(reservationColVal); + activeReservations.put(fateId, reservation); + }); } catch (TableNotFoundException e) { throw new IllegalStateException(tableName + " not found!", e); } + + return activeReservations; + } + + @Override + public void deleteDeadReservations() { + for (Entry entry : getActiveReservations().entrySet()) { + FateId fateId = entry.getKey(); + FateReservation reservation = entry.getValue(); + if (isDeadReservation(reservation)) { + newMutator(fateId).putUnreserveTx(reservation).tryMutate(); + // No need to check the status... If it is ACCEPTED, we have successfully unreserved + // the dead transaction. If it is REJECTED, the reservation has changed (i.e., + // has been unreserved so no need to do anything, or has been unreserved and reserved + // again in which case we don't want to change it). If it is UNKNOWN, the mutation + // may or may not have been written. If it was written, we have successfully unreserved + // the dead transaction. If it was not written, the next cycle/call to + // deleteDeadReservations() will try again. + } + } } @Override @@ -268,7 +308,7 @@ protected Pair> getStatusAndKey(FateId fateId) { @Override protected FateTxStore newUnreservedFateTxStore(FateId fateId) { - return new FateTxStoreImpl(fateId, false); + return new FateTxStoreImpl(fateId); } static Range getRow(FateId fateId) { @@ -297,16 +337,13 @@ public FateInstanceType type() { } private class FateTxStoreImpl extends AbstractFateTxStoreImpl { - private boolean isReserved; - private FateTxStoreImpl(FateId fateId, boolean isReserved) { + private FateTxStoreImpl(FateId fateId) { super(fateId); - this.isReserved = isReserved; } - @Override - protected boolean isReserved() { - return isReserved; + private FateTxStoreImpl(FateId fateId, FateReservation reservation) { + super(fateId, reservation); } @Override @@ -450,13 +487,13 @@ private Optional findTop() { @Override protected void unreserve() { - if (!this.deleted) { - FateMutator.Status status = newMutator(fateId).putUnreserveTx(fateId).tryMutate(); - if (!status.equals(FateMutator.Status.ACCEPTED)) { - throw new IllegalStateException("Failed to unreserve " + fateId); - } + if (!deleted) { + FateMutator.Status status; + do { + status = newMutator(fateId).putUnreserveTx(reservation).tryMutate(); + } while (status.equals(FateMutator.Status.UNKNOWN)); } - this.isReserved = false; + reservation = null; } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java b/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java index 07777687574..012e2853ff2 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/schema/FateSchema.java @@ -36,8 +36,8 @@ public static class TxColumnFamily { public static final String CREATE_TIME = "ctime"; public static final ColumnFQ CREATE_TIME_COLUMN = new ColumnFQ(NAME, new Text(CREATE_TIME)); - public static final String RESERVED = "reserved"; - public static final ColumnFQ RESERVED_COLUMN = new ColumnFQ(NAME, new Text(RESERVED)); + public static final String RESERVATION = "reservation"; + public static final ColumnFQ RESERVATION_COLUMN = new ColumnFQ(NAME, new Text(RESERVATION)); } public static class TxInfoColumnFamily { diff --git a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java index a3c1b9cfd90..f6c32caa8e5 100644 --- a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java +++ b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.logging; import java.io.Serializable; +import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; @@ -169,6 +170,26 @@ public Optional> createAndReserve(FateKey fateKey) { } return txStore; } + + @Override + public boolean isReserved(FateId fateId) { + return store.isReserved(fateId); + } + + @Override + public Map getActiveReservations() { + return store.getActiveReservations(); + } + + @Override + public void deleteDeadReservations() { + store.deleteDeadReservations(); + } + + @Override + public boolean isDeadReservation(FateReservation reservation) { + return store.isDeadReservation(reservation); + } }; } } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index db2d7da7701..3c2c6461626 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -82,6 +82,28 @@ public Optional> tryReserve(FateId fateId) { } } + public boolean isReserved(FateId fateId) { + return reserved.contains(fateId); + } + + @Override + public Map getActiveReservations() { + // This method only makes sense for the FateStores that don't store their reservations in memory + throw new UnsupportedOperationException(); + } + + @Override + public void deleteDeadReservations() { + // This method only makes sense for the FateStores that don't store their reservations in memory + throw new UnsupportedOperationException(); + } + + @Override + public boolean isDeadReservation(FateReservation reservation) { + // This method only makes sense for the FateStores that don't store their reservations in memory + throw new UnsupportedOperationException(); + } + private class TestFateTxStore implements FateTxStore { private final FateId fateId; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index 7dbf36ab522..2e6487439d7 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import java.io.BufferedWriter; import java.io.File; @@ -62,7 +63,6 @@ import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.manager.thrift.TFateId; @@ -772,10 +772,9 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); - // TODO 4131 dummy lock for now MetaFateStore mfs = - new MetaFateStore<>(fateZkPath, zk, new ZooUtil.LockID("path", "node", 1234)); - UserFateStore ufs = new UserFateStore<>(context); + new MetaFateStore<>(fateZkPath, zk, context.getZooCache(), createDummyLockID()); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID()); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); Map> readOnlyFateStores = diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index 866cb81c70d..7424d873c7b 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -1085,9 +1085,9 @@ boolean canSuspendTablets() { try { var metaInstance = initializeFateInstance(context, new MetaFateStore<>(getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter(), managerLock.getLockID())); + context.getZooReaderWriter(), context.getZooCache(), managerLock.getLockID())); var userInstance = initializeFateInstance(context, - new UserFateStore<>(context, AccumuloTable.FATE.tableName())); + new UserFateStore<>(context, AccumuloTable.FATE.tableName(), managerLock.getLockID())); if (!fateRefs.compareAndSet(null, Map.of(FateInstanceType.META, metaInstance, FateInstanceType.USER, userInstance))) { @@ -1199,6 +1199,7 @@ private Fate initializeFateInstance(ServerContext context, FateStore fateInstance = new Fate<>(this, store, TraceRepo::toLogString, getConfiguration()); + fateInstance.startDeadReservationCleaner(); var fateCleaner = new FateCleaner<>(store, Duration.ofHours(8), System::nanoTime); ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java index 3240aef0b6a..0ade4b71cfd 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.manager.metrics.fate; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; + import java.util.Map.Entry; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -27,7 +29,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.metrics.MetricsUtil; import org.apache.accumulo.core.util.threads.ThreadPools; @@ -72,9 +73,8 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - // TODO 4131 dummy lock for now this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter(), - new ZooUtil.LockID("path", "node", 1234)); + context.getZooCache(), createDummyLockID()); } catch (KeeperException ex) { throw new IllegalStateException( "FATE Metrics - Failed to create zoo store - metrics unavailable", ex); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java index a36266e6c9c..3c5f035f943 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java @@ -19,6 +19,7 @@ package org.apache.accumulo.manager.upgrade; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING; import static org.apache.accumulo.server.AccumuloDataVersion.REMOVE_DEPRECATIONS_FOR_VERSION_3; import static org.apache.accumulo.server.AccumuloDataVersion.ROOT_TABLET_META_CHANGES; @@ -40,7 +41,6 @@ import org.apache.accumulo.core.conf.ConfigCheckUtil; import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.volume.Volume; @@ -308,10 +308,9 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - // TODO 4131 dummy lock for now final ReadOnlyFateStore fate = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter(), new ZooUtil.LockID("path", "node", 1234)); + context.getZooReaderWriter(), context.getZooCache(), createDummyLockID()); try (var idStream = fate.list()) { if (idStream.findFirst().isPresent()) { throw new AccumuloException("Aborting upgrade because there are" diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index 3f1fe9c55a4..635e810e90f 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.compaction; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP1; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP2; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP3; @@ -31,7 +32,6 @@ import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.row; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.verify; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.writeData; -import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.apache.accumulo.test.util.FileMetadataUtil.countFencedFiles; import static org.apache.accumulo.test.util.FileMetadataUtil.splitFilesIntoRanges; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -234,7 +234,7 @@ public void testExternalCompaction() throws Exception { public void testCompactionCommitAndDeadDetectionRoot() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), createTestLockID()); + ctx.getZooReaderWriter(), ctx.getZooCache(), createDummyLockID()); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var tableId = ctx.getTableId(AccumuloTable.ROOT.tableName()); @@ -253,7 +253,7 @@ public void testCompactionCommitAndDeadDetectionRoot() throws Exception { public void testCompactionCommitAndDeadDetectionMeta() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), createTestLockID()); + ctx.getZooReaderWriter(), ctx.getZooCache(), createDummyLockID()); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { // Metadata table by default already has 2 tablets @@ -275,7 +275,7 @@ public void testCompactionCommitAndDeadDetectionUser() throws Exception { final String tableName = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { - UserFateStore userFateStore = new UserFateStore<>(ctx); + UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID()); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); c.tableOperations().create(tableName, new NewTableConfiguration().withSplits(splits)); @@ -298,9 +298,10 @@ public void testCompactionCommitAndDeadDetectionAll() throws Exception { final String userTable = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { - UserFateStore userFateStore = new UserFateStore<>(ctx); - FateStore metaFateStore = new MetaFateStore<>( - ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), createTestLockID()); + UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID()); + FateStore metaFateStore = + new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), + ctx.getZooCache(), createDummyLockID()); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java index 00847ad5f78..2bd54477e41 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java @@ -492,7 +492,9 @@ protected Fate initializeFate(FateStore store) { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - return new Fate<>(new TestEnv(), store, r -> r + "", config); + Fate fate = new Fate<>(new TestEnv(), store, r -> r + "", config); + fate.startDeadReservationCleaner(); + return fate; } protected abstract TStatus getTxStatus(ServerContext sctx, FateId fateId); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java index 5bddd4f35c1..39fda418243 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java @@ -552,7 +552,9 @@ private Fate initializeFate(FateStore store) { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - return new Fate<>(new TestEnv(), store, Object::toString, config); + Fate fate = new Fate<>(new TestEnv(), store, Object::toString, config); + fate.startDeadReservationCleaner(); + return fate; } private boolean wordIsTStatus(String word) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index f5f88fdf05a..378c7bb896e 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -21,21 +21,25 @@ import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.conf.DefaultConfiguration; -import org.apache.accumulo.core.fate.AbstractFateStore; +import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; @@ -47,19 +51,27 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.apache.accumulo.test.util.Wait; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; +import org.apache.hadoop.shaded.org.mockito.Mockito; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.collect.Sets; // TODO 4131 could potentially have separate classes for testing MetaFateStore and UserFateStore +// similar to how FateTestRunner is used, however that interface doesn't work as nicely here +// since we are using multiple stores instead of just one. Can do something similar to +// FateTestRunner here if desired public class MultipleStoresIT extends SharedMiniClusterBase { + private static final Logger LOG = LoggerFactory.getLogger(MultipleStoresIT.class); @TempDir private static File tempDir; private static ZooKeeperTestingServer szk = null; @@ -96,28 +108,25 @@ public void testReserveUnreserve() throws Exception { testReserveUnreserve(FateInstanceType.USER); } - protected void testReserveUnreserve(FateInstanceType storeType) throws Exception { + private void testReserveUnreserve(FateInstanceType storeType) throws Exception { // reserving/unreserving a FateId should be reflected across instances of the stores - String tableName = getUniqueNames(1)[0]; - List expReservedList = new ArrayList<>(); + final String tableName = getUniqueNames(1)[0]; final int numFateIds = 500; - List> reservations = new ArrayList<>(); - boolean isUserStore = storeType.equals(FateInstanceType.USER); - Set allIds = new HashSet<>(); - final AbstractFateStore store1, store2; + final List> reservations = new ArrayList<>(); + final boolean isUserStore = storeType.equals(FateInstanceType.USER); + final Set allIds = new HashSet<>(); + final FateStore store1, store2; + final ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + final ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + Map activeReservations; if (isUserStore) { createFateTable(client, tableName); - } - - if (isUserStore) { - store1 = new UserFateStore<>(client, tableName); - store2 = new UserFateStore<>(client, tableName); + store1 = new UserFateStore<>(client, tableName, lock1); + store2 = new UserFateStore<>(client, tableName, lock2); } else { - ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); - ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); - store1 = new MetaFateStore<>(FATE_DIR, zk, lock1); - store2 = new MetaFateStore<>(FATE_DIR, zk, lock2); + store1 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock1); + store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); } // Create the fate ids using store1 @@ -131,7 +140,6 @@ protected void testReserveUnreserve(FateInstanceType storeType) throws Exception // ids were created using store1 int count = 0; for (FateId fateId : allIds) { - expReservedList.add(fateId); if (count % 2 == 0) { reservations.add(store1.reserve(fateId)); assertTrue(store2.tryReserve(fateId).isEmpty()); @@ -141,11 +149,11 @@ protected void testReserveUnreserve(FateInstanceType storeType) throws Exception } count++; } - // Both stores should return the same list of reserved transactions - assertTrue(expReservedList.containsAll(store1.getReservedTxns()) - && expReservedList.size() == store1.getReservedTxns().size()); - assertTrue(expReservedList.containsAll(store2.getReservedTxns()) - && expReservedList.size() == store2.getReservedTxns().size()); + // Both stores should return the same reserved transactions + activeReservations = store1.getActiveReservations(); + assertEquals(allIds, activeReservations.keySet()); + activeReservations = store2.getActiveReservations(); + assertEquals(allIds, activeReservations.keySet()); // Test setting/getting the TStatus and unreserving the transactions for (int i = 0; i < allIds.size(); i++) { @@ -156,15 +164,38 @@ protected void testReserveUnreserve(FateInstanceType storeType) throws Exception reservation.delete(); reservation.unreserve(0, TimeUnit.MILLISECONDS); // Attempt to set a status on a tx that has been unreserved (should throw exception) - try { - reservation.setStatus(ReadOnlyFateStore.TStatus.NEW); - fail(); - } catch (Exception e) { - // Expected - } + assertThrows(IllegalStateException.class, + () -> reservation.setStatus(ReadOnlyFateStore.TStatus.NEW)); } - assertEquals(List.of(), store1.getReservedTxns()); - assertEquals(List.of(), store2.getReservedTxns()); + assertTrue(store1.getActiveReservations().isEmpty()); + assertTrue(store2.getActiveReservations().isEmpty()); + } + + @Test + public void testReserveNonExistentTxn() throws Exception { + testReserveNonExistentTxn(FateInstanceType.META); + testReserveNonExistentTxn(FateInstanceType.USER); + } + + private void testReserveNonExistentTxn(FateInstanceType storeType) throws Exception { + // Tests that reserve() doesn't hang indefinitely and instead throws an error + // on reserve() a non-existent transaction. Tests that tryReserve() will return + // an empty optional on non-existent transaction. + final FateStore store; + final boolean isUserStore = storeType.equals(FateInstanceType.USER); + final String tableName = getUniqueNames(1)[0]; + final FateId fakeFateId = FateId.from(storeType, UUID.randomUUID()); + final ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); + + if (isUserStore) { + createFateTable(client, tableName); + store = new UserFateStore<>(client, tableName, lock); + } else { + store = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock); + } + + assertThrows(IllegalStateException.class, () -> store.reserve(fakeFateId)); + assertTrue(store.tryReserve(fakeFateId).isEmpty()); } @Test @@ -173,24 +204,21 @@ public void testReserveReservedAndUnreserveUnreserved() throws Exception { testReserveReservedAndUnreserveUnreserved(FateInstanceType.USER); } - public void testReserveReservedAndUnreserveUnreserved(FateInstanceType storeType) + private void testReserveReservedAndUnreserveUnreserved(FateInstanceType storeType) throws Exception { - String tableName = getUniqueNames(1)[0]; + final String tableName = getUniqueNames(1)[0]; final int numFateIds = 500; - boolean isUserStore = storeType.equals(FateInstanceType.USER); - Set allIds = new HashSet<>(); - List> reservations = new ArrayList<>(); - final AbstractFateStore store; + final boolean isUserStore = storeType.equals(FateInstanceType.USER); + final Set allIds = new HashSet<>(); + final List> reservations = new ArrayList<>(); + final FateStore store; + final ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); if (isUserStore) { createFateTable(client, tableName); - } - - if (isUserStore) { - store = new UserFateStore<>(client, tableName); + store = new UserFateStore<>(client, tableName, lock); } else { - ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); - store = new MetaFateStore<>(FATE_DIR, zk, lock); + store = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock); } // Create some FateIds and ensure that they can be reserved @@ -214,12 +242,8 @@ public void testReserveReservedAndUnreserveUnreserved(FateInstanceType storeType } // Try to unreserve again (should throw exception) for (var reservation : reservations) { - try { - reservation.unreserve(0, TimeUnit.MILLISECONDS); - fail(); - } catch (Exception e) { - // Expected - } + assertThrows(IllegalStateException.class, + () -> reservation.unreserve(0, TimeUnit.MILLISECONDS)); } } @@ -229,24 +253,21 @@ public void testReserveAfterUnreserveAndReserveAfterDeleted() throws Exception { testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType.USER); } - public void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType storeType) + private void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType storeType) throws Exception { - String tableName = getUniqueNames(1)[0]; + final String tableName = getUniqueNames(1)[0]; final int numFateIds = 500; - boolean isUserStore = storeType.equals(FateInstanceType.USER); - Set allIds = new HashSet<>(); - List> reservations = new ArrayList<>(); - final AbstractFateStore store; + final boolean isUserStore = storeType.equals(FateInstanceType.USER); + final Set allIds = new HashSet<>(); + final List> reservations = new ArrayList<>(); + final FateStore store; + final ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); if (isUserStore) { createFateTable(client, tableName); - } - - if (isUserStore) { - store = new UserFateStore<>(client, tableName); + store = new UserFateStore<>(client, tableName, lock); } else { - ZooUtil.LockID lock = new ZooUtil.LockID("/locks", "L1", 50); - store = new MetaFateStore<>(FATE_DIR, zk, lock); + store = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock); } // Create some FateIds and ensure that they can be reserved @@ -278,12 +299,7 @@ public void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType sto // Verify that the tx is now unknown since it has been deleted assertEquals(ReadOnlyFateStore.TStatus.UNKNOWN, store.read(fateId).getStatus()); // Attempt to reserve a deleted txn, should throw an exception and not wait indefinitely - try { - store.reserve(fateId); - fail(); - } catch (Exception e) { - // Expected - } + assertThrows(IllegalStateException.class, () -> store.reserve(fateId)); } } @@ -293,43 +309,42 @@ public void testMultipleFateInstances() throws Exception { testMultipleFateInstances(FateInstanceType.USER); } - public void testMultipleFateInstances(FateInstanceType storeType) throws Exception { - String tableName = getUniqueNames(1)[0]; + private void testMultipleFateInstances(FateInstanceType storeType) throws Exception { + final String tableName = getUniqueNames(1)[0]; final int numFateIds = 500; - boolean isUserStore = storeType.equals(FateInstanceType.USER); - Set allIds = new HashSet<>(); - final AbstractFateStore store1, store2; - final TestEnv testEnv1 = new TestEnv(); - final TestEnv testEnv2 = new TestEnv(); + final boolean isUserStore = storeType.equals(FateInstanceType.USER); + final Set allIds = new HashSet<>(); + final FateStore store1, store2; + final SleepingTestEnv testEnv1 = new SleepingTestEnv(50); + final SleepingTestEnv testEnv2 = new SleepingTestEnv(50); + final ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + final ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); if (isUserStore) { createFateTable(client, tableName); - } - - if (isUserStore) { - store1 = new UserFateStore<>(client, tableName); - store2 = new UserFateStore<>(client, tableName); + store1 = new UserFateStore<>(client, tableName, lock1); + store2 = new UserFateStore<>(client, tableName, lock2); } else { - ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); - ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); - store1 = new MetaFateStore<>(FATE_DIR, zk, lock1); - store2 = new MetaFateStore<>(FATE_DIR, zk, lock2); + store1 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock1); + store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); } - Fate fate1 = + Fate fate1 = new Fate<>(testEnv1, store1, Object::toString, DefaultConfiguration.getInstance()); - Fate fate2 = + fate1.startDeadReservationCleaner(); + Fate fate2 = new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); + fate2.startDeadReservationCleaner(); for (int i = 0; i < numFateIds; i++) { FateId fateId; // Start half the txns using fate1, and the other half using fate2 if (i % 2 == 0) { fateId = fate1.startTransaction(); - fate1.seedTransaction("op" + i, fateId, new TestRepo(), true, "test"); + fate1.seedTransaction("op" + i, fateId, new SleepingTestRepo(), true, "test"); } else { fateId = fate2.startTransaction(); - fate2.seedTransaction("op" + i, fateId, new TestRepo(), true, "test"); + fate2.seedTransaction("op" + i, fateId, new SleepingTestRepo(), true, "test"); } allIds.add(fateId); } @@ -347,11 +362,124 @@ public void testMultipleFateInstances(FateInstanceType storeType) throws Excepti fate2.shutdown(1, TimeUnit.MINUTES); } - public static class TestRepo implements Repo { + @Test + public void testDeadReservationsCleanup() throws Exception { + testDeadReservationsCleanup(FateInstanceType.META); + testDeadReservationsCleanup(FateInstanceType.USER); + } + + private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exception { + // Tests reserving some transactions, then simulating that the Manager died by creating + // a new Fate instance and store with a new LockID. The transactions which were + // reserved using the old LockID should be cleaned up by Fate's DeadReservationCleaner, + // then picked up by the new Fate/store. + + final String tableName = getUniqueNames(1)[0]; + // One transaction for each FATE worker thread + final int numFateIds = + Integer.parseInt(Property.MANAGER_FATE_THREADPOOL_SIZE.getDefaultValue()); + final boolean isUserStore = storeType.equals(FateInstanceType.USER); + final Set allIds = new HashSet<>(); + final FateStore store1, store2, spyStore1; + final LatchTestEnv testEnv1 = new LatchTestEnv(); + final LatchTestEnv testEnv2 = new LatchTestEnv(); + final ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + final ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + Map reservations; + + if (isUserStore) { + createFateTable(client, tableName); + store1 = new UserFateStore<>(client, tableName, lock1); + } else { + store1 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock1); + } + + // Redefine isDeadReservation() for store1 as always being false. We don't want fate1/store1 to + // delete any reservations yet (we are simulating that the Manager is alive right now) + spyStore1 = Mockito.spy(store1); + Mockito.doAnswer(invocation -> false).when(spyStore1) + .isDeadReservation(Mockito.any(FateStore.FateReservation.class)); + + Fate fate1 = + new Fate<>(testEnv1, spyStore1, Object::toString, DefaultConfiguration.getInstance()); + fate1.startDeadReservationCleaner(); + + // Ensure nothing is reserved yet + assertTrue(spyStore1.getActiveReservations().isEmpty()); + + // Create transactions + for (int i = 0; i < numFateIds; i++) { + FateId fateId; + fateId = fate1.startTransaction(); + fate1.seedTransaction("op" + i, fateId, new LatchTestRepo(), true, "test"); + allIds.add(fateId); + } + assertEquals(numFateIds, allIds.size()); + + // Wait for all the fate worker threads to start working on the transactions + Wait.waitFor(() -> testEnv1.numWorkers.get() == numFateIds); + // Each fate worker will be hung up working (IN_PROGRESS) on a single transaction + + // Verify spyStore1 has the transactions reserved and that they were reserved with lock1 + reservations = spyStore1.getActiveReservations(); + assertEquals(allIds, reservations.keySet()); + reservations.values().forEach( + res -> assertTrue(FateStore.FateReservation.locksAreEqual(lock1, res.getLockID()))); + + if (isUserStore) { + store2 = new UserFateStore<>(client, tableName, lock2); + } else { + store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); + } + + // Simulate what would happen if the Manager using the Fate object (fate1) died. + // ServerLock.isLockHeld(...) would return false for the LockId of the Manager that died + // (in this case, lock1). + + // Redefine what is considered "dead" as those whose locks match lock1 + Mockito.doAnswer(invocation -> { + FateStore.FateReservation reservation = invocation.getArgument(0); + return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); + }).when(spyStore1).isDeadReservation(Mockito.any(FateStore.FateReservation.class)); + + // Verify store2 can see the reserved transactions even though they were reserved using + // spyStore1 + reservations = store2.getActiveReservations(); + assertEquals(allIds, reservations.keySet()); + reservations.values().forEach( + res -> assertTrue(FateStore.FateReservation.locksAreEqual(lock1, res.getLockID()))); + + // Create the new Fate/start the Fate threads (the work finder and the workers). + // The DeadReservationCleaner for fate2 should not run/have no effect since we + // already have a DeadReservationCleaner for storeType running from fate1. + Fate fate2 = + new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); + fate2.startDeadReservationCleaner(); + + // Wait for the "dead" reservations to be deleted and picked up again (reserved using + // fate2/store2/lock2 now). + // They are considered "dead" if they are held by lock1 in this test. We don't have to worry + // about fate1/spyStore1/lock1 being used to reserve the transactions again since all + // the workers for fate1 are hung up + Wait.waitFor(() -> { + Map store2Reservations = store2.getActiveReservations(); + boolean allReservedWithLock2 = store2Reservations.values().stream() + .allMatch(entry -> FateStore.FateReservation.locksAreEqual(entry.getLockID(), lock2)); + return store2Reservations.keySet().equals(allIds) && allReservedWithLock2; + }, 60_000); + + // Finish work and shutdown + testEnv1.workersLatch.countDown(); + testEnv2.workersLatch.countDown(); + fate1.shutdown(1, TimeUnit.MINUTES); + fate2.shutdown(1, TimeUnit.MINUTES); + } + + public static class SleepingTestRepo implements Repo { private static final long serialVersionUID = 1L; @Override - public long isReady(FateId fateId, TestEnv environment) { + public long isReady(FateId fateId, SleepingTestEnv environment) { return 0; } @@ -361,14 +489,17 @@ public String getName() { } @Override - public Repo call(FateId fateId, TestEnv environment) throws Exception { + public Repo call(FateId fateId, SleepingTestEnv environment) throws Exception { environment.executedOps.add(fateId); - Thread.sleep(50); // Simulate some work + LOG.debug("Thread " + Thread.currentThread() + " in SleepingTestRepo.call() sleeping for " + + environment.sleepTimeMs + " millis"); + Thread.sleep(environment.sleepTimeMs); // Simulate some work + LOG.debug("Thread " + Thread.currentThread() + " finished SleepingTestRepo.call()"); return null; } @Override - public void undo(FateId fateId, TestEnv environment) { + public void undo(FateId fateId, SleepingTestEnv environment) { } @@ -378,7 +509,51 @@ public String getReturn() { } } - public static class TestEnv { + public static class SleepingTestEnv { public final Set executedOps = Collections.synchronizedSet(new HashSet<>()); + public final int sleepTimeMs; + + public SleepingTestEnv(int sleepTimeMs) { + this.sleepTimeMs = sleepTimeMs; + } + } + + public static class LatchTestRepo implements Repo { + private static final long serialVersionUID = 1L; + + @Override + public long isReady(FateId fateId, LatchTestEnv environment) { + return 0; + } + + @Override + public String getName() { + return null; + } + + @Override + public Repo call(FateId fateId, LatchTestEnv environment) throws Exception { + LOG.debug("Thread " + Thread.currentThread() + " in LatchTestRepo.call()"); + environment.numWorkers.incrementAndGet(); + environment.workersLatch.await(); + LOG.debug("Thread " + Thread.currentThread() + " finished LatchTestRepo.call()"); + environment.numWorkers.decrementAndGet(); + return null; + } + + @Override + public void undo(FateId fateId, LatchTestEnv environment) { + + } + + @Override + public String getReturn() { + return null; + } + } + + public static class LatchTestEnv { + public final AtomicInteger numWorkers = new AtomicInteger(0); + public final CountDownLatch workersLatch = new CountDownLatch(1); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java index ddb30d253a5..37aecd358dd 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -34,7 +35,6 @@ import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateIT; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; @@ -76,14 +76,10 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, createTestLockID(), + testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, createDummyLockID(), maxDeferred, fateIdGenerator), sctx); } - public static ZooUtil.LockID createTestLockID() { - return new ZooUtil.LockID("S1", "N1", 1234); - } - @Override protected TStatus getTxStatus(ServerContext sctx, FateId fateId) { try { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java index 70c9bdccc6b..84cfbb7d1f1 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java @@ -18,7 +18,7 @@ */ package org.apache.accumulo.test.fate.meta; -import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore; @@ -34,6 +34,7 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, ServerContext sctx = getCluster().getServerContext(); String path = sctx.getZooKeeperRoot() + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); - testMethod.execute(new MetaFateStore<>(path, zk, createTestLockID()), sctx); + testMethod.execute(new MetaFateStore<>(path, zk, sctx.getZooCache(), createDummyLockID()), + sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java index c21c387275d..9ad75f84911 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java @@ -18,8 +18,8 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; -import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; @@ -29,11 +29,13 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.Method; +import java.util.Optional; import java.util.UUID; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.MetaFateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; @@ -76,8 +78,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, - createTestLockID(), maxDeferred, fateIdGenerator); + MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, + createDummyLockID(), maxDeferred, fateIdGenerator); // Check that the store has no transactions before and after each test assertEquals(0, store.list().count()); @@ -90,39 +92,53 @@ protected void deleteKey(FateId fateId, ServerContext sctx) { try { // We have to use reflection since the NodeValue is internal to the store - // Grab both the constructors that use the serialized bytes and status, lock, uuid + // Grab both the constructors that use the serialized bytes and status, reservation Class nodeClass = Class.forName(MetaFateStore.class.getName() + "$NodeValue"); - Constructor statusLockUUIDCons = - nodeClass.getDeclaredConstructor(TStatus.class, String.class, String.class); + Constructor statusReservationCons = + nodeClass.getDeclaredConstructor(TStatus.class, FateStore.FateReservation.class); Constructor serializedCons = nodeClass.getDeclaredConstructor(byte[].class); - statusLockUUIDCons.setAccessible(true); + statusReservationCons.setAccessible(true); serializedCons.setAccessible(true); - // Get the status, lock, and uuid fields so they can be read and the serialize method + // Get the status and reservation fields so they can be read and get the serialize method Field nodeStatus = nodeClass.getDeclaredField("status"); - Field nodeLock = nodeClass.getDeclaredField("lockID"); - Field nodeUUID = nodeClass.getDeclaredField("uuid"); + Field nodeReservation = nodeClass.getDeclaredField("reservation"); Method nodeSerialize = nodeClass.getDeclaredMethod("serialize"); nodeStatus.setAccessible(true); - nodeLock.setAccessible(true); - nodeUUID.setAccessible(true); + nodeReservation.setAccessible(true); nodeSerialize.setAccessible(true); - // Get the existing status, lock, and uuid for the node and build a new node with an empty key + // Get the existing status and reservation for the node and build a new node with an empty key // but uses the existing tid String txPath = ZK_ROOT + Constants.ZFATE + "/tx_" + fateId.getTxUUIDStr(); Object currentNode = serializedCons.newInstance(new Object[] {zk.getData(txPath)}); TStatus currentStatus = (TStatus) nodeStatus.get(currentNode); - String currentLock = (String) nodeLock.get(currentNode); - String currentUUID = (String) nodeUUID.get(currentNode); - // replace the node with no key and just a tid and existing status, lock, and uuid - Object newNode = statusLockUUIDCons.newInstance(currentStatus, currentLock, currentUUID); + Optional currentReservation = + getCurrentReservation(nodeReservation, currentNode); + // replace the node with no key and just a tid and existing status and reservation + Object newNode = + statusReservationCons.newInstance(currentStatus, currentReservation.orElse(null)); - // Replace the transaction with the same status, lock, and uuid and no key + // Replace the transaction with the same status and reservation but no key zk.putPersistentData(txPath, (byte[]) nodeSerialize.invoke(newNode), NodeExistsPolicy.OVERWRITE); } catch (Exception e) { throw new IllegalStateException(e); } } + + private Optional getCurrentReservation(Field nodeReservation, + Object currentNode) throws Exception { + Object currentResAsObject = nodeReservation.get(currentNode); + Optional currentReservation = Optional.empty(); + if (currentResAsObject instanceof Optional) { + Optional currentResAsOptional = (Optional) currentResAsObject; + if (currentResAsOptional.isPresent() + && currentResAsOptional.orElseThrow() instanceof FateStore.FateReservation) { + currentReservation = + Optional.of((FateStore.FateReservation) currentResAsOptional.orElseThrow()); + } + } + return currentReservation; + } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java index aaa9be01f16..0a1e6ae4d03 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java @@ -33,8 +33,11 @@ import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; +import org.apache.accumulo.core.fate.user.FateMutator; import org.apache.accumulo.core.fate.user.FateMutatorImpl; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.test.fate.FateIT; import org.junit.jupiter.api.AfterAll; @@ -169,6 +172,100 @@ public void requireStatus() throws Exception { } + @Test + public void testReservations() throws Exception { + final String table = getUniqueNames(1)[0]; + try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { + client.tableOperations().create(table, ntc); + + ClientContext context = (ClientContext) client; + + FateId fateId = + FateId.from(FateInstanceType.fromNamespaceOrTableName(table), UUID.randomUUID()); + ZooUtil.LockID lockID = new ZooUtil.LockID("/locks", "L1", 50); + FateStore.FateReservation reservation = + FateStore.FateReservation.from(lockID, UUID.randomUUID()); + FateStore.FateReservation wrongReservation = + FateStore.FateReservation.from(lockID, UUID.randomUUID()); + + // Ensure that we cannot do anything in the column until it is initialized + FateMutator.Status status = + new FateMutatorImpl<>(context, table, fateId).putReservedTx(reservation).tryMutate(); + assertEquals(REJECTED, status); + status = + new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); + assertEquals(REJECTED, status); + status = + new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); + assertEquals(REJECTED, status); + + // Initialize the column and ensure we can't do it twice + status = new FateMutatorImpl<>(context, table, fateId).putInitReserveColVal().tryMutate(); + assertEquals(ACCEPTED, status); + status = new FateMutatorImpl<>(context, table, fateId).putInitReserveColVal().tryMutate(); + assertEquals(REJECTED, status); + + // Ensure that reserving is the only thing we can do + status = + new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); + assertEquals(REJECTED, status); + status = + new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); + assertEquals(REJECTED, status); + // It is considered unreserved since it has been initialized + status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); + assertEquals(ACCEPTED, status); + + // Should be able to reserve + status = new FateMutatorImpl<>(context, table, fateId).putReservedTx(reservation).tryMutate(); + assertEquals(ACCEPTED, status); + + // Ensure that it is reserved + status = + new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); + assertEquals(ACCEPTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireReserved(wrongReservation) + .tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); + assertEquals(ACCEPTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); + assertEquals(REJECTED, status); + // Should not be able to reserve when it is already reserved + status = + new FateMutatorImpl<>(context, table, fateId).putReservedTx(wrongReservation).tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).putReservedTx(reservation).tryMutate(); + assertEquals(REJECTED, status); + + // Should be able to unreserve + status = new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(wrongReservation) + .tryMutate(); + assertEquals(REJECTED, status); + status = + new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); + assertEquals(ACCEPTED, status); + status = + new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); + assertEquals(REJECTED, status); + + // Ensure that it is unreserved + status = + new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); + assertEquals(REJECTED, status); + status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); + assertEquals(ACCEPTED, status); + } + } + void logAllEntriesInTable(String tableName, AccumuloClient client) throws Exception { client.createScanner(tableName).forEach(e -> log.info(e.getKey() + " " + e.getValue())); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java index 4d6a8da1188..49d310cc1e5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; import java.util.stream.StreamSupport; @@ -60,7 +61,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute(new UserFateStore<>(client, table, maxDeferred, fateIdGenerator), + testMethod.execute( + new UserFateStore<>(client, table, createDummyLockID(), maxDeferred, fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java index fde93de3460..db3341ecfa3 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; + import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.test.fate.FateOpsCommandsIT; @@ -26,7 +28,7 @@ public class UserFateOpsCommandsIT extends FateOpsCommandsIT { @Override public void executeTest(FateTestExecutor testMethod, int maxDeferred, AbstractFateStore.FateIdGenerator fateIdGenerator) throws Exception { - testMethod.execute(new UserFateStore<>(getCluster().getServerContext()), + testMethod.execute(new UserFateStore<>(getCluster().getServerContext(), createDummyLockID()), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java index 2a3248f7671..8cb66cb43b8 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.fate.user.UserFateStore.getRowId; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; @@ -56,7 +57,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute(new UserFateStore<>(client, table, maxDeferred, fateIdGenerator), + testMethod.execute( + new UserFateStore<>(client, table, createDummyLockID(), maxDeferred, fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java index 129d5632568..7efbe676b2a 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -86,7 +87,7 @@ private static class TestUserFateStore extends UserFateStore { // use the list of fateIds to simulate collisions on fateIds public TestUserFateStore(ClientContext context, String tableName, List fateIds) { - super(context, tableName); + super(context, tableName, createDummyLockID()); this.fateIdIterator = fateIds.iterator(); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index 210220b8cb1..7153d2a1248 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -21,7 +21,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -263,11 +263,12 @@ public void getFateStatus() { InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - MetaFateStore mfs = new MetaFateStore<>( - ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createTestLockID()); + MetaFateStore mfs = + new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, + context.getZooCache(), createDummyLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); - UserFateStore ufs = new UserFateStore<>(context); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID()); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); @@ -358,7 +359,7 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); MetaFateStore mfs = new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, - zk, createTestLockID()); + zk, context.getZooCache(), createDummyLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); AdminUtil.FateStatus fateStatus = admin.getStatus(mfs, zk, lockPath, null, null, null); @@ -388,8 +389,8 @@ private boolean lookupFateInAccumulo(final String tableName) throws KeeperExcept log.trace("tid: {}", tableId); - UserFateStore as = new UserFateStore<>(context); - AdminUtil.FateStatus fateStatus = admin.getStatus(as, null, null, null); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID()); + AdminUtil.FateStatus fateStatus = admin.getStatus(ufs, null, null, null); log.trace("current fates: {}", fateStatus.getTransactions().size()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java index 825d12ed3d9..bb3d29fc6bf 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java @@ -19,8 +19,8 @@ package org.apache.accumulo.test.functional; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FLUSH_ID; -import static org.apache.accumulo.test.fate.meta.MetaFateIT.createTestLockID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -232,8 +232,8 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore mfs = - new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, createTestLockID()); + MetaFateStore mfs = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, + zk, context.getZooCache(), createDummyLockID()); var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); return admin.getStatus(mfs, zk, lockPath, null, null, null); } catch (KeeperException | InterruptedException e) { From 47d16b09e6ade2402b72ddac3b01b5897a9b7015 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Mon, 3 Jun 2024 12:55:40 -0400 Subject: [PATCH 03/17] Changes: Formatting, fixed a test failure occurring with a newly added test that was merged in (logic error in my existing code) --- .../org/apache/accumulo/core/fate/FateStore.java | 2 +- .../apache/accumulo/core/fate/MetaFateStore.java | 4 ---- .../accumulo/test/fate/MultipleStoresIT.java | 16 +++++++--------- .../test/fate/meta/MetaFateInterleavingIT.java | 4 +++- .../test/fate/user/UserFateInterleavingIT.java | 4 ++-- 5 files changed, 13 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index f11c16587fc..2f31569e8ef 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.Serializable; import java.io.UncheckedIOException; +import java.time.Duration; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -31,7 +32,6 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.hadoop.io.DataInputBuffer; -import java.time.Duration; /** * Transaction Store: a place to save transactions diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index 0a58137b812..c3aad75c4a1 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -121,10 +121,6 @@ protected void create(FateId fateId, FateKey key) { @Override public Optional> tryReserve(FateId fateId) { - // return an empty option if the FateId doesn't exist - if (_getStatus(fateId).equals(TStatus.UNKNOWN)) { - return Optional.empty(); - } // uniquely identify this attempt to reserve the fate operation data FateReservation reservation = FateReservation.from(lockID, UUID.randomUUID()); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index 378c7bb896e..a7559fd29a9 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -25,6 +25,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -162,7 +163,7 @@ private void testReserveUnreserve(FateInstanceType storeType) throws Exception { reservation.setStatus(ReadOnlyFateStore.TStatus.SUBMITTED); assertEquals(ReadOnlyFateStore.TStatus.SUBMITTED, reservation.getStatus()); reservation.delete(); - reservation.unreserve(0, TimeUnit.MILLISECONDS); + reservation.unreserve(Duration.ofMillis(0)); // Attempt to set a status on a tx that has been unreserved (should throw exception) assertThrows(IllegalStateException.class, () -> reservation.setStatus(ReadOnlyFateStore.TStatus.NEW)); @@ -179,8 +180,7 @@ public void testReserveNonExistentTxn() throws Exception { private void testReserveNonExistentTxn(FateInstanceType storeType) throws Exception { // Tests that reserve() doesn't hang indefinitely and instead throws an error - // on reserve() a non-existent transaction. Tests that tryReserve() will return - // an empty optional on non-existent transaction. + // on reserve() a non-existent transaction. final FateStore store; final boolean isUserStore = storeType.equals(FateInstanceType.USER); final String tableName = getUniqueNames(1)[0]; @@ -195,7 +195,6 @@ private void testReserveNonExistentTxn(FateInstanceType storeType) throws Except } assertThrows(IllegalStateException.class, () -> store.reserve(fakeFateId)); - assertTrue(store.tryReserve(fakeFateId).isEmpty()); } @Test @@ -238,12 +237,11 @@ private void testReserveReservedAndUnreserveUnreserved(FateInstanceType storeTyp // Unreserve all the FateIds for (var reservation : reservations) { - reservation.unreserve(0, TimeUnit.MILLISECONDS); + reservation.unreserve(Duration.ofMillis(0)); } // Try to unreserve again (should throw exception) for (var reservation : reservations) { - assertThrows(IllegalStateException.class, - () -> reservation.unreserve(0, TimeUnit.MILLISECONDS)); + assertThrows(IllegalStateException.class, () -> reservation.unreserve(Duration.ofMillis(0))); } } @@ -282,7 +280,7 @@ private void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType st // Unreserve all for (var reservation : reservations) { - reservation.unreserve(0, TimeUnit.MILLISECONDS); + reservation.unreserve(Duration.ofMillis(0)); } // Ensure they can be reserved again, and delete and unreserve this time @@ -292,7 +290,7 @@ private void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType st var reservation = store.tryReserve(fateId); assertFalse(reservation.isEmpty()); reservation.orElseThrow().delete(); - reservation.orElseThrow().unreserve(0, TimeUnit.MILLISECONDS); + reservation.orElseThrow().unreserve(Duration.ofMillis(0)); } for (FateId fateId : allIds) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java index 0ba14f730e2..64dedafed3f 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java @@ -39,6 +39,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred String path = ZK_ROOT + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); zk.mkdirs(ZK_ROOT); - testMethod.execute(new MetaFateStore<>(path, zk), sctx); + testMethod.execute( + new MetaFateStore<>(path, zk, sctx.getZooCache(), AbstractFateStore.createDummyLockID()), + sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java index afd4d8ac5aa..c5fa134cc66 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java @@ -34,8 +34,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute(new UserFateStore<>(client, table, maxDeferred, fateIdGenerator), - getCluster().getServerContext()); + testMethod.execute(new UserFateStore<>(client, table, AbstractFateStore.createDummyLockID(), + maxDeferred, fateIdGenerator), getCluster().getServerContext()); client.tableOperations().delete(table); } } From 0809fd399fce27e35d73e518ef9cf34b2e4eb90e Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Mon, 3 Jun 2024 14:01:29 -0400 Subject: [PATCH 04/17] Changed a method call which is available in hadoop 3.3.6 but not in the hadoop version used in the github build (3.0.3), causing a build failure --- .../java/org/apache/accumulo/test/fate/MultipleStoresIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index a7559fd29a9..a38f0a67e1e 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -436,7 +436,7 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce // Redefine what is considered "dead" as those whose locks match lock1 Mockito.doAnswer(invocation -> { - FateStore.FateReservation reservation = invocation.getArgument(0); + FateStore.FateReservation reservation = invocation.getArgument(0, FateStore.FateReservation.class); return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); }).when(spyStore1).isDeadReservation(Mockito.any(FateStore.FateReservation.class)); From ee6bb8847bb982d8b6ca859eba144360a6167b03 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Mon, 3 Jun 2024 14:18:46 -0400 Subject: [PATCH 05/17] formatting --- .../java/org/apache/accumulo/core/fate/user/UserFateStore.java | 1 + .../java/org/apache/accumulo/test/fate/MultipleStoresIT.java | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index 6c9ea4364dd..42305334471 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -224,6 +224,7 @@ public void deleteDeadReservations() { } } + @Override protected Stream getTransactions(Set statuses) { try { Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index a38f0a67e1e..82b46a03c28 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -436,7 +436,8 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce // Redefine what is considered "dead" as those whose locks match lock1 Mockito.doAnswer(invocation -> { - FateStore.FateReservation reservation = invocation.getArgument(0, FateStore.FateReservation.class); + FateStore.FateReservation reservation = + invocation.getArgument(0, FateStore.FateReservation.class); return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); }).when(spyStore1).isDeadReservation(Mockito.any(FateStore.FateReservation.class)); From a92095d53695544a16064e2469011cf27c569b4f Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Mon, 3 Jun 2024 15:18:18 -0400 Subject: [PATCH 06/17] Changed a method call which is available in hadoop 3.3.6 but not in the hadoop version used in the github build (3.0.3), causing a build failure. This was not fixed with 0809fd399fce27e35d73e518ef9cf34b2e4eb90e. Should be fixed now. --- .../java/org/apache/accumulo/test/fate/MultipleStoresIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index 82b46a03c28..13780b6624b 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -437,7 +437,7 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce // Redefine what is considered "dead" as those whose locks match lock1 Mockito.doAnswer(invocation -> { FateStore.FateReservation reservation = - invocation.getArgument(0, FateStore.FateReservation.class); + (FateStore.FateReservation) invocation.getArguments()[0]; return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); }).when(spyStore1).isDeadReservation(Mockito.any(FateStore.FateReservation.class)); From be1ac103cc754456668193412ce5ccd3c0cbb4bb Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 4 Jun 2024 14:13:08 -0400 Subject: [PATCH 07/17] Mockito -> EasyMock in MultipleStoresIT Changed MultipleStoresIT from using a shaded library in hadoop (Mockito) to using EasyMock --- .../accumulo/test/fate/MultipleStoresIT.java | 69 +++++++++++-------- 1 file changed, 41 insertions(+), 28 deletions(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index 13780b6624b..4a43601ca68 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -19,6 +19,9 @@ package org.apache.accumulo.test.fate; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -49,12 +52,13 @@ import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.user.UserFateStore; +import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.test.util.Wait; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; -import org.apache.hadoop.shaded.org.mockito.Mockito; +import org.easymock.EasyMock; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -378,7 +382,7 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce Integer.parseInt(Property.MANAGER_FATE_THREADPOOL_SIZE.getDefaultValue()); final boolean isUserStore = storeType.equals(FateInstanceType.USER); final Set allIds = new HashSet<>(); - final FateStore store1, store2, spyStore1; + final FateStore mockedStore1, store2; final LatchTestEnv testEnv1 = new LatchTestEnv(); final LatchTestEnv testEnv2 = new LatchTestEnv(); final ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); @@ -387,23 +391,29 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce if (isUserStore) { createFateTable(client, tableName); - store1 = new UserFateStore<>(client, tableName, lock1); + mockedStore1 = EasyMock.createMockBuilder(UserFateStore.class) + .withConstructor(ClientContext.class, String.class, ZooUtil.LockID.class) + .withArgs(client, tableName, lock1).addMockedMethod("isDeadReservation").createMock(); } else { - store1 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock1); - } - - // Redefine isDeadReservation() for store1 as always being false. We don't want fate1/store1 to - // delete any reservations yet (we are simulating that the Manager is alive right now) - spyStore1 = Mockito.spy(store1); - Mockito.doAnswer(invocation -> false).when(spyStore1) - .isDeadReservation(Mockito.any(FateStore.FateReservation.class)); + mockedStore1 = EasyMock.createMockBuilder(MetaFateStore.class) + .withConstructor(String.class, ZooReaderWriter.class, ZooCache.class, + ZooUtil.LockID.class) + .withArgs(FATE_DIR, zk, client.getZooCache(), lock1).addMockedMethod("isDeadReservation") + .createMock(); + } + // Define isDeadReservation() for mockedStore1 as always being false. We don't want + // fate1/mockedStore1 to delete any reservations yet (we are simulating that the + // Manager is alive right now) + expect(mockedStore1.isDeadReservation(anyObject(FateStore.FateReservation.class))) + .andReturn(false).anyTimes(); + replay(mockedStore1); Fate fate1 = - new Fate<>(testEnv1, spyStore1, Object::toString, DefaultConfiguration.getInstance()); + new Fate<>(testEnv1, mockedStore1, Object::toString, DefaultConfiguration.getInstance()); fate1.startDeadReservationCleaner(); // Ensure nothing is reserved yet - assertTrue(spyStore1.getActiveReservations().isEmpty()); + assertTrue(mockedStore1.getActiveReservations().isEmpty()); // Create transactions for (int i = 0; i < numFateIds; i++) { @@ -418,8 +428,8 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce Wait.waitFor(() -> testEnv1.numWorkers.get() == numFateIds); // Each fate worker will be hung up working (IN_PROGRESS) on a single transaction - // Verify spyStore1 has the transactions reserved and that they were reserved with lock1 - reservations = spyStore1.getActiveReservations(); + // Verify mockedStore1 has the transactions reserved and that they were reserved with lock1 + reservations = mockedStore1.getActiveReservations(); assertEquals(allIds, reservations.keySet()); reservations.values().forEach( res -> assertTrue(FateStore.FateReservation.locksAreEqual(lock1, res.getLockID()))); @@ -430,24 +440,27 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); } - // Simulate what would happen if the Manager using the Fate object (fate1) died. - // ServerLock.isLockHeld(...) would return false for the LockId of the Manager that died - // (in this case, lock1). - - // Redefine what is considered "dead" as those whose locks match lock1 - Mockito.doAnswer(invocation -> { - FateStore.FateReservation reservation = - (FateStore.FateReservation) invocation.getArguments()[0]; - return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); - }).when(spyStore1).isDeadReservation(Mockito.any(FateStore.FateReservation.class)); - // Verify store2 can see the reserved transactions even though they were reserved using - // spyStore1 + // mockedStore1 reservations = store2.getActiveReservations(); assertEquals(allIds, reservations.keySet()); reservations.values().forEach( res -> assertTrue(FateStore.FateReservation.locksAreEqual(lock1, res.getLockID()))); + // Simulate what would happen if the Manager using the Fate object (fate1) died. + // ServerLock.isLockHeld(...) would return false for the LockId of the Manager that died + // (in this case, lock1). + + // Redefine what is considered "dead" as those whose locks match lock1 + EasyMock.reset(mockedStore1); + expect(mockedStore1.isDeadReservation(anyObject(FateStore.FateReservation.class))) + .andAnswer(() -> { + FateStore.FateReservation reservation = + (FateStore.FateReservation) EasyMock.getCurrentArguments()[0]; + return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); + }).anyTimes(); + replay(mockedStore1); + // Create the new Fate/start the Fate threads (the work finder and the workers). // The DeadReservationCleaner for fate2 should not run/have no effect since we // already have a DeadReservationCleaner for storeType running from fate1. @@ -458,7 +471,7 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce // Wait for the "dead" reservations to be deleted and picked up again (reserved using // fate2/store2/lock2 now). // They are considered "dead" if they are held by lock1 in this test. We don't have to worry - // about fate1/spyStore1/lock1 being used to reserve the transactions again since all + // about fate1/mockedStore1/lock1 being used to reserve the transactions again since all // the workers for fate1 are hung up Wait.waitFor(() -> { Map store2Reservations = store2.getActiveReservations(); From 51a709390a6e171713a52573f0ba77fd563654fe Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 2 Jul 2024 09:47:17 -0400 Subject: [PATCH 08/17] Addressed review comments: - Simplified MetaFateStore.isReserved(FateId) (unnecessary error handling) - Added check in MetaFateStore.FateTxStoreImpl.setStatus() that was missing: needed to ensure that the reservation stored in the store and in ZK were equivalent - Added some logging to MetaFateStore.FateTxStoreImpl.unreserve() - No longer using a ConditionalMutation to check if a FateId is reserved for UserFateStore, just scan the table instead - Made the DeadReservationCleaner a critical thread to the Manager (if it dies, so does the Manager) - Stores now take a Predicate isLockHeld which is used to determine if the given LockID is live. This considerably simplified the new MultipleStoresIT and the stores themselves. --- .../accumulo/core/fate/AbstractFateStore.java | 23 ++-- .../org/apache/accumulo/core/fate/Fate.java | 111 ++++++++---------- .../apache/accumulo/core/fate/FateStore.java | 10 +- .../accumulo/core/fate/MetaFateStore.java | 49 ++++---- .../fate/user/ColumnValueMappingIterator.java | 71 ----------- .../accumulo/core/fate/user/FateMutator.java | 23 ---- .../core/fate/user/FateMutatorImpl.java | 22 ---- .../fate/user/ReservationMappingIterator.java | 77 ------------ .../core/fate/user/StatusMappingIterator.java | 44 ++++++- .../core/fate/user/UserFateStore.java | 44 +++++-- .../accumulo/core/logging/FateLogger.java | 5 - .../apache/accumulo/core/fate/TestStore.java | 6 - .../apache/accumulo/server/util/Admin.java | 6 +- .../org/apache/accumulo/manager/Manager.java | 10 +- .../manager/metrics/fate/FateMetrics.java | 5 +- .../manager/upgrade/UpgradeCoordinator.java | 6 +- .../compaction/ExternalCompaction_1_IT.java | 14 +-- .../org/apache/accumulo/test/fate/FateIT.java | 4 +- .../accumulo/test/fate/FateOpsCommandsIT.java | 4 +- .../accumulo/test/fate/MultipleStoresIT.java | 97 ++++++--------- .../accumulo/test/fate/meta/MetaFateIT.java | 5 +- .../fate/meta/MetaFateInterleavingIT.java | 4 +- .../test/fate/meta/MetaFateOpsCommandsIT.java | 5 +- .../test/fate/meta/MetaFateStoreFateIT.java | 5 +- .../test/fate/user/FateMutatorImplIT.java | 37 ------ .../accumulo/test/fate/user/UserFateIT.java | 3 +- .../fate/user/UserFateInterleavingIT.java | 5 +- .../test/fate/user/UserFateOpsCommandsIT.java | 4 +- .../test/fate/user/UserFateStoreFateIT.java | 3 +- .../test/fate/user/UserFateStoreIT.java | 3 +- .../test/functional/FateConcurrencyIT.java | 12 +- .../test/functional/FunctionalTestUtils.java | 5 +- 32 files changed, 234 insertions(+), 488 deletions(-) delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index e723ba4a64f..a3cbf829605 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -40,12 +40,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import java.util.function.Predicate; import java.util.stream.Stream; import org.apache.accumulo.core.fate.Fate.TxInfo; -import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.time.NanoTime; @@ -78,7 +77,7 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { // The ZooKeeper lock for the process that's running this store instance protected final ZooUtil.LockID lockID; - protected final ZooCache zooCache; + protected final Predicate isLockHeld; protected final Map deferred; private final int maxDeferred; private final AtomicBoolean deferredOverflow = new AtomicBoolean(); @@ -94,17 +93,18 @@ public AbstractFateStore() { this(null, null, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } - public AbstractFateStore(ZooUtil.LockID lockID, ZooCache zooCache) { - this(lockID, zooCache, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + public AbstractFateStore(ZooUtil.LockID lockID, Predicate isLockHeld) { + this(lockID, isLockHeld, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } - public AbstractFateStore(ZooUtil.LockID lockID, ZooCache zooCache, int maxDeferred, - FateIdGenerator fateIdGenerator) { + public AbstractFateStore(ZooUtil.LockID lockID, Predicate isLockHeld, + int maxDeferred, FateIdGenerator fateIdGenerator) { this.maxDeferred = maxDeferred; this.fateIdGenerator = Objects.requireNonNull(fateIdGenerator); this.deferred = Collections.synchronizedMap(new HashMap<>()); - this.lockID = lockID; - this.zooCache = zooCache; + this.lockID = Objects.requireNonNullElseGet(lockID, AbstractFateStore::createDummyLockID); + // If the store is used for a Fate object, this should be non-null, otherwise null is fine + this.isLockHeld = isLockHeld; } public static byte[] serialize(Object o) { @@ -346,11 +346,6 @@ public Optional> createAndReserve(FateKey fateKey) { return txStore; } - @Override - public boolean isDeadReservation(FateReservation reservation) { - return !ServiceLock.isLockHeld(zooCache, reservation.getLockID()); - } - protected abstract void create(FateId fateId, FateKey fateKey); protected abstract Pair> getStatusAndKey(FateId fateId); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index aa044cadff4..fa68825267d 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -37,6 +37,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedTransferQueue; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -73,7 +75,8 @@ public class Fate { private final FateStore store; private final T environment; private final ScheduledThreadPoolExecutor fatePoolWatcher; - private final ExecutorService executor; + private final ExecutorService transactionExecutor; + private final ExecutorService deadResCleanerExecutor; private static final EnumSet FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); private static boolean userDeadReservationCleanerRunning = false; @@ -82,8 +85,6 @@ public class Fate { private final AtomicBoolean keepRunning = new AtomicBoolean(true); private final TransferQueue workQueue; private final Thread workFinder; - // Will be null if this Fate instance is not running a DeadReservationCleaner - private Thread deadReservationCleaner; public enum TxInfo { TX_NAME, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE @@ -334,15 +335,10 @@ protected Repo executeCall(FateId fateId, Repo op) throws Exception { * reservations for {@link MetaFateStore}). */ private class DeadReservationCleaner implements Runnable { - // TODO 4131 periodic check runs every 30 seconds - // Should this be longer? Shorter? A configurable Property? A function of something? - private static final long INTERVAL_MILLIS = 30_000; - @Override public void run() { - while (keepRunning.get()) { + if (keepRunning.get()) { store.deleteDeadReservations(); - UtilWaitThread.sleep(INTERVAL_MILLIS); } } } @@ -384,37 +380,30 @@ public Fate(T environment, FateStore store, Function,String> toLogStr } } }, 3, SECONDS)); - this.executor = pool; - - this.workFinder = Threads.createThread("Fate work finder", new WorkFinder()); - this.workFinder.start(); - } + this.transactionExecutor = pool; - /** - * Starts a thread that periodically cleans up "dead reservations" (see - * {@link FateStore#deleteDeadReservations()}). Only one thread is started per store type - * ({@link FateInstanceType}) for subsequent calls to this method. - */ - public void startDeadReservationCleaner() { - // TODO 4131 this is not ideal starting this thread in its own start method, but the other - // threads in the constructor. However, starting this thread in the constructor causes - // a Maven build failure, and do not want to move starting the other threads into a - // method in this PR... should be done standalone (see issue#4609). - - if ((store.type().equals(FateInstanceType.USER) && !userDeadReservationCleanerRunning) - || store.type().equals(FateInstanceType.META) && !metaDeadReservationCleanerRunning) { - if (store.type().equals(FateInstanceType.USER)) { - this.deadReservationCleaner = - Threads.createThread("USER dead reservation cleaner", new DeadReservationCleaner()); + // Create a dead reservation cleaner for this store that will periodically (every 30 seconds) + // clean up reservations held by dead processes, if they exist. Only created if a dead + // reservation cleaner is not already running for the given store type. + // TODO 4131 periodic cleanup runs every 30 seconds + // Should this be longer? Shorter? A configurable Property? A function of something? + ScheduledExecutorService deadResCleanerExecutor = ThreadPools.getServerThreadPools() + .createScheduledExecutorService(1, store.type() + "-dead-reservation-cleaner-pool"); + if ((store.type() == FateInstanceType.USER && !userDeadReservationCleanerRunning) + || (store.type() == FateInstanceType.META && !metaDeadReservationCleanerRunning)) { + ScheduledFuture deadReservationCleaner = deadResCleanerExecutor + .scheduleWithFixedDelay(new DeadReservationCleaner(), 3, 30, SECONDS); + ThreadPools.watchCriticalScheduledTask(deadReservationCleaner); + if (store.type() == FateInstanceType.USER) { userDeadReservationCleanerRunning = true; - } else if (store.type().equals(FateInstanceType.META)) { - this.deadReservationCleaner = - Threads.createThread("META dead reservation cleaner", new DeadReservationCleaner()); + } else if (store.type() == FateInstanceType.META) { metaDeadReservationCleanerRunning = true; } - this.deadReservationCleaner.start(); } + this.deadResCleanerExecutor = deadResCleanerExecutor; + this.workFinder = Threads.createThread("Fate work finder", new WorkFinder()); + this.workFinder.start(); } // get a transaction id back to the requester before doing any work @@ -581,65 +570,57 @@ public Stream list(FateKey.FateKeyType type) { public void shutdown(long timeout, TimeUnit timeUnit) { if (keepRunning.compareAndSet(true, false)) { fatePoolWatcher.shutdown(); - executor.shutdown(); + transactionExecutor.shutdown(); workFinder.interrupt(); - if (deadReservationCleaner != null) { - deadReservationCleaner.interrupt(); - } + deadResCleanerExecutor.shutdown(); } if (timeout > 0) { long start = System.nanoTime(); while ((System.nanoTime() - start) < timeUnit.toNanos(timeout) && (workFinder.isAlive() - || (deadReservationCleaner != null && deadReservationCleaner.isAlive()) - || !executor.isTerminated())) { + || !transactionExecutor.isTerminated() || !deadResCleanerExecutor.isTerminated())) { try { - if (!executor.awaitTermination(1, SECONDS)) { + if (!transactionExecutor.awaitTermination(1, SECONDS)) { log.debug("Fate {} is waiting for worker threads to terminate", store.type()); continue; } + if (!deadResCleanerExecutor.awaitTermination(1, SECONDS)) { + log.debug("Fate {} is waiting for dead reservation cleaner thread to terminate", + store.type()); + continue; + } + workFinder.join(1_000); if (workFinder.isAlive()) { log.debug("Fate {} is waiting for work finder thread to terminate", store.type()); workFinder.interrupt(); } - - if (deadReservationCleaner != null) { - deadReservationCleaner.join(1_000); - } - if (deadReservationCleaner != null && deadReservationCleaner.isAlive()) { - log.debug("Fate {} is waiting for dead reservation cleaner thread to terminate", - store.type()); - deadReservationCleaner.interrupt(); - } } catch (InterruptedException e) { throw new RuntimeException(e); } } - if (workFinder.isAlive() - || (deadReservationCleaner != null && deadReservationCleaner.isAlive()) - || !executor.isTerminated()) { + if (workFinder.isAlive() || !transactionExecutor.isTerminated() + || !deadResCleanerExecutor.isTerminated()) { log.warn( - "Waited for {}ms for all fate {} background threads to stop, but some are still running. workFinder:{} deadReservationCleaner:{} executor:{}", + "Waited for {}ms for all fate {} background threads to stop, but some are still running. workFinder:{} transactionExecutor:{} deadResCleanerExecutor:{}", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), store.type(), - workFinder.isAlive(), - (deadReservationCleaner != null && deadReservationCleaner.isAlive()), - !executor.isTerminated()); + workFinder.isAlive(), !transactionExecutor.isTerminated(), + !deadResCleanerExecutor.isTerminated()); } } + // interrupt the background threads + transactionExecutor.shutdownNow(); + deadResCleanerExecutor.shutdownNow(); + // Update that USER/META dead reservation cleaner is no longer running - if (deadReservationCleaner != null && !deadReservationCleaner.isAlive()) { - if (store.type().equals(FateInstanceType.USER)) { - userDeadReservationCleanerRunning = false; - } else if (store.type().equals(FateInstanceType.META)) { - metaDeadReservationCleanerRunning = false; - } + if (store.type() == FateInstanceType.USER && userDeadReservationCleanerRunning) { + userDeadReservationCleanerRunning = false; + } else if (store.type() == FateInstanceType.META && metaDeadReservationCleanerRunning) { + metaDeadReservationCleanerRunning = false; } - // interrupt the background threads - executor.shutdownNow(); } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 2f31569e8ef..5640dae1f7e 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -175,7 +175,7 @@ public static FateReservation from(String fateReservationStr) { /** * - * @param fateReservationStr the string from a call to FateReservations toString() + * @param fateReservationStr the string from a call to {@link FateReservation#toString()} * @return true if the string represents a valid FateReservation object, false otherwise */ public static boolean isFateReservation(String fateReservationStr) { @@ -261,14 +261,6 @@ public int hashCode() { */ void deleteDeadReservations(); - /** - * The way dead reservations are determined for {@link #deleteDeadReservations()} - * - * @param reservation the fate reservation - * @return true if reservation held by a dead Manager, false otherwise - */ - boolean isDeadReservation(FateReservation reservation); - /** * Attempt to reserve the fate transaction. * diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index f03279cb2a3..4cf95419fa5 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -36,12 +36,12 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; +import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Stream; import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.fate.Fate.TxInfo; -import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; @@ -72,16 +72,16 @@ private String getTXPath(FateId fateId) { return path + "/tx_" + fateId.getTxUUIDStr(); } - public MetaFateStore(String path, ZooReaderWriter zk, ZooCache zooCache, ZooUtil.LockID lockID) - throws KeeperException, InterruptedException { - this(path, zk, zooCache, lockID, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID, + Predicate isLockHeld) throws KeeperException, InterruptedException { + this(path, zk, lockID, isLockHeld, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } @VisibleForTesting - public MetaFateStore(String path, ZooReaderWriter zk, ZooCache zooCache, ZooUtil.LockID lockID, - int maxDeferred, FateIdGenerator fateIdGenerator) + public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID, + Predicate isLockHeld, int maxDeferred, FateIdGenerator fateIdGenerator) throws KeeperException, InterruptedException { - super(lockID, zooCache, maxDeferred, fateIdGenerator); + super(lockID, isLockHeld, maxDeferred, fateIdGenerator); this.path = path; this.zk = zk; @@ -152,14 +152,7 @@ public Optional> tryReserve(FateId fateId) { @Override public boolean isReserved(FateId fateId) { - boolean isReserved; - try { - isReserved = getNode(fateId).isReserved(); - } catch (Exception e) { - // Exception thrown, so node doesn't exist, so it is not reserved - isReserved = false; - } - return isReserved; + return getNode(fateId).isReserved(); } @Override @@ -193,7 +186,7 @@ public void deleteDeadReservations() { // Make sure the current node is still reserved and reserved with the expected reservation // and it is dead if (currNodeVal.isReserved() && currNodeVal.reservation.orElseThrow().equals(reservation) - && isDeadReservation(currNodeVal.reservation.orElseThrow())) { + && !isLockHeld.test(currNodeVal.reservation.orElseThrow().getLockID())) { // Delete the reservation return new NodeValue(currNodeVal.status, null, currNodeVal.fateKey.orElse(null)) .serialize(); @@ -325,10 +318,17 @@ public void setStatus(TStatus status) { try { zk.mutateExisting(getTXPath(fateId), currSerializedData -> { NodeValue currNodeVal = new NodeValue(currSerializedData); - FateReservation currFateReservation = currNodeVal.reservation.orElse(null); - FateKey currFateKey = currNodeVal.fateKey.orElse(null); - NodeValue newNodeValue = new NodeValue(status, currFateReservation, currFateKey); - return newNodeValue.serialize(); + // Ensure the FateId is reserved in ZK, and it is reserved with the expected reservation + if (currNodeVal.isReserved() + && currNodeVal.reservation.orElseThrow().equals(this.reservation)) { + FateReservation currFateReservation = currNodeVal.reservation.orElseThrow(); + FateKey currFateKey = currNodeVal.fateKey.orElse(null); + NodeValue newNodeValue = new NodeValue(status, currFateReservation, currFateKey); + return newNodeValue.serialize(); + } else { + throw new IllegalStateException("Either the FateId is not reserved in ZK, or it is" + + " but the reservation in ZK differs from that in the store."); + } }); } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { throw new IllegalStateException(e); @@ -428,13 +428,18 @@ protected void unreserve() { zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { NodeValue currNodeVal = new NodeValue(currSerNodeVal); FateKey currFateKey = currNodeVal.fateKey.orElse(null); - if ((currNodeVal.isReserved() - && currNodeVal.reservation.orElseThrow().equals(this.reservation))) { + if (currNodeVal.isReserved() + && currNodeVal.reservation.orElseThrow().equals(this.reservation)) { // Remove the FateReservation from the NodeValue to unreserve return new NodeValue(currNodeVal.status, null, currFateKey).serialize(); } else { // possible this is running a 2nd time in zk server fault conditions and its first // write went through + if (!currNodeVal.isReserved()) { + log.trace("The FATE reservation does not exist in ZK"); + } else if (!currNodeVal.reservation.orElseThrow().equals(this.reservation)) { + log.debug("The FATE reservation in ZK differs from that in the store"); + } return null; } }); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java deleted file mode 100644 index 1d73608350a..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/ColumnValueMappingIterator.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.accumulo.core.fate.user; - -import java.io.IOException; -import java.util.Collection; -import java.util.Objects; - -import org.apache.accumulo.core.data.ByteSequence; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Range; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.IteratorEnvironment; -import org.apache.accumulo.core.iterators.SortedKeyValueIterator; - -public abstract class ColumnValueMappingIterator implements SortedKeyValueIterator { - - protected SortedKeyValueIterator source; - protected Value mappedValue; - - protected abstract void mapValue(); - - @Override - public boolean hasTop() { - return source.hasTop(); - } - - @Override - public void next() throws IOException { - source.next(); - mapValue(); - } - - @Override - public void seek(Range range, Collection columnFamilies, boolean inclusive) - throws IOException { - source.seek(range, columnFamilies, inclusive); - mapValue(); - } - - @Override - public Key getTopKey() { - return source.getTopKey(); - } - - @Override - public Value getTopValue() { - return Objects.requireNonNull(mappedValue); - } - - @Override - public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { - throw new UnsupportedOperationException(); - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java index e06d8638eac..416a45ff317 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java @@ -59,29 +59,6 @@ public interface FateMutator { */ FateMutator putInitReserveColVal(); - /** - * Require that the transaction is reserved with a specific {@link FateStore.FateReservation} - * - * @param reservation the reservation - * @return the FateMutator with the added condition - */ - FateMutator requireReserved(FateStore.FateReservation reservation); - - /** - * Require that the transaction is reserved (can be reserved with any - * {@link FateStore.FateReservation}) - * - * @return the FateMutator with the added condition - */ - FateMutator requireReserved(); - - /** - * Require that the transaction is unreserved - * - * @return the FateMutator with the added condition - */ - FateMutator requireUnreserved(); - FateMutator putName(byte[] data); FateMutator putAutoClean(byte[] data); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java index 550e21755ab..c85f530a868 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java @@ -109,28 +109,6 @@ public FateMutator putInitReserveColVal() { return this; } - @Override - public FateMutator requireReserved(FateStore.FateReservation reservation) { - Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()).setValue(reservation.toString()); - mutation.addCondition(condition); - return this; - } - - @Override - public FateMutator requireReserved() { - Condition condition = ReservationMappingIterator.createRequireReservedCondition(); - mutation.addCondition(condition); - return this; - } - - @Override - public FateMutator requireUnreserved() { - Condition condition = ReservationMappingIterator.createRequireUnreservedCondition(); - mutation.addCondition(condition); - return this; - } - @Override public FateMutator putName(byte[] data) { TxInfoColumnFamily.TX_NAME_COLUMN.put(mutation, new Value(data)); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java deleted file mode 100644 index 8a04c435f14..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/ReservationMappingIterator.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.accumulo.core.fate.user; - -import static org.apache.accumulo.core.fate.user.schema.FateSchema.TxColumnFamily.RESERVATION_COLUMN; - -import java.io.IOException; -import java.util.Map; - -import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.data.Condition; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.fate.FateStore; -import org.apache.accumulo.core.iterators.IteratorEnvironment; -import org.apache.accumulo.core.iterators.SortedKeyValueIterator; - -/** - * An iterator used for determining if the reservation column for a FateId has a FateReservation set - * or not. Maps the value of the column to "isreserved" or "notreserved" if the column has a - * FateReservation value set or not. - */ -public class ReservationMappingIterator extends ColumnValueMappingIterator { - - private static final String IS_RESERVED = "isreserved"; - private static final String NOT_RESERVED = "notreserved"; - - @Override - public void init(SortedKeyValueIterator source, Map options, - IteratorEnvironment env) throws IOException { - this.source = source; - // No need for options or env - } - - @Override - protected void mapValue() { - if (hasTop()) { - String currVal = source.getTopValue().toString(); - mappedValue = FateStore.FateReservation.isFateReservation(currVal) ? new Value(IS_RESERVED) - : new Value(NOT_RESERVED); - } else { - mappedValue = null; - } - } - - public static Condition createRequireReservedCondition() { - Condition condition = new Condition(RESERVATION_COLUMN.getColumnFamily(), - RESERVATION_COLUMN.getColumnQualifier()); - IteratorSetting is = new IteratorSetting(100, ReservationMappingIterator.class); - - return condition.setValue(IS_RESERVED).setIterators(is); - } - - public static Condition createRequireUnreservedCondition() { - Condition condition = new Condition(RESERVATION_COLUMN.getColumnFamily(), - RESERVATION_COLUMN.getColumnQualifier()); - IteratorSetting is = new IteratorSetting(100, ReservationMappingIterator.class); - - return condition.setValue(NOT_RESERVED).setIterators(is); - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java index d9c667b25b4..1a0fae5aa3e 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/StatusMappingIterator.java @@ -22,14 +22,18 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Condition; import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.iterators.IteratorEnvironment; @@ -40,13 +44,15 @@ * iterator allows for checking of the status column's value against a set of acceptable statuses * within a conditional mutation. */ -public class StatusMappingIterator extends ColumnValueMappingIterator { +public class StatusMappingIterator implements SortedKeyValueIterator { private static final String PRESENT = "present"; private static final String ABSENT = "absent"; private static final String STATUS_SET_KEY = "statusSet"; + private SortedKeyValueIterator source; private final Set acceptableStatuses = new HashSet<>(); + private Value mappedValue; /** * The set of acceptable must be provided as an option to the iterator using the @@ -64,12 +70,29 @@ public void init(SortedKeyValueIterator source, Map op } } + @Override + public boolean hasTop() { + return source.hasTop(); + } + + @Override + public void next() throws IOException { + source.next(); + mapValue(); + } + + @Override + public void seek(Range range, Collection columnFamilies, boolean inclusive) + throws IOException { + source.seek(range, columnFamilies, inclusive); + mapValue(); + } + /** * Maps the value of the status column to "present" or "absent" based on its presence within the * set of statuses. */ - @Override - protected void mapValue() { + private void mapValue() { if (source.hasTop()) { String currentValue = source.getTopValue().toString(); mappedValue = @@ -79,6 +102,21 @@ protected void mapValue() { } } + @Override + public Key getTopKey() { + return source.getTopKey(); + } + + @Override + public Value getTopValue() { + return Objects.requireNonNull(mappedValue); + } + + @Override + public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { + throw new UnsupportedOperationException(); + } + /** * Creates a condition that checks if the status column's value is one of the given acceptable * statuses. diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index 42305334471..42f5e5c45f3 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.UUID; import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -73,20 +74,22 @@ public class UserFateStore extends AbstractFateStore { private static final com.google.common.collect.Range REPO_RANGE = com.google.common.collect.Range.closed(1, maxRepos); - public UserFateStore(ClientContext context, String tableName, ZooUtil.LockID lockID) { - this(context, tableName, lockID, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + public UserFateStore(ClientContext context, String tableName, ZooUtil.LockID lockID, + Predicate isLockHeld) { + this(context, tableName, lockID, isLockHeld, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } @VisibleForTesting public UserFateStore(ClientContext context, String tableName, ZooUtil.LockID lockID, - int maxDeferred, FateIdGenerator fateIdGenerator) { - super(lockID, context.getZooCache(), maxDeferred, fateIdGenerator); + Predicate isLockHeld, int maxDeferred, FateIdGenerator fateIdGenerator) { + super(lockID, isLockHeld, maxDeferred, fateIdGenerator); this.context = Objects.requireNonNull(context); this.tableName = Objects.requireNonNull(tableName); } - public UserFateStore(ClientContext context, ZooUtil.LockID lockID) { - this(context, AccumuloTable.FATE.tableName(), lockID); + public UserFateStore(ClientContext context, ZooUtil.LockID lockID, + Predicate isLockHeld) { + this(context, AccumuloTable.FATE.tableName(), lockID, isLockHeld); } @Override @@ -170,9 +173,19 @@ public Optional> tryReserve(FateId fateId) { // we can return the FateTxStore since it was successfully reserved in this // attempt, otherwise we return empty (was written by another reservation // attempt or was not written at all). - status = newMutator(fateId).requireReserved(reservation).tryMutate(); - if (status.equals(FateMutator.Status.ACCEPTED)) { - return Optional.of(new FateTxStoreImpl(fateId, reservation)); + try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { + scanner.setRange(getRow(fateId)); + scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); + FateReservation persistedRes = scanner.stream() + .filter(entry -> FateReservation.isFateReservation(entry.getValue().toString())) + .map(entry -> FateReservation.from(entry.getValue().toString())).findFirst() + .orElse(null); + if (persistedRes != null && persistedRes.equals(reservation)) { + return Optional.of(new FateTxStoreImpl(fateId, reservation)); + } + } catch (TableNotFoundException e) { + throw new IllegalStateException(tableName + " not found!", e); } } return Optional.empty(); @@ -180,7 +193,16 @@ public Optional> tryReserve(FateId fateId) { @Override public boolean isReserved(FateId fateId) { - return newMutator(fateId).requireReserved().tryMutate().equals(FateMutator.Status.ACCEPTED); + try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { + scanner.setRange(getRow(fateId)); + scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); + return scanner.stream() + .map(entry -> FateReservation.isFateReservation(entry.getValue().toString())).findFirst() + .orElse(false); + } catch (TableNotFoundException e) { + throw new IllegalStateException(tableName + " not found!", e); + } } @Override @@ -211,7 +233,7 @@ public void deleteDeadReservations() { for (Entry entry : getActiveReservations().entrySet()) { FateId fateId = entry.getKey(); FateReservation reservation = entry.getValue(); - if (isDeadReservation(reservation)) { + if (!isLockHeld.test(reservation.getLockID())) { newMutator(fateId).putUnreserveTx(reservation).tryMutate(); // No need to check the status... If it is ACCEPTED, we have successfully unreserved // the dead transaction. If it is REJECTED, the reservation has changed (i.e., diff --git a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java index 8a44f03f937..2acc6e41fac 100644 --- a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java +++ b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java @@ -191,11 +191,6 @@ public Map getActiveReservations() { public void deleteDeadReservations() { store.deleteDeadReservations(); } - - @Override - public boolean isDeadReservation(FateReservation reservation) { - return store.isDeadReservation(reservation); - } }; } } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 3ad7ad6ce6e..632fba88685 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -94,12 +94,6 @@ public void deleteDeadReservations() { throw new UnsupportedOperationException(); } - @Override - public boolean isDeadReservation(FateReservation reservation) { - // This method only makes sense for the FateStores that don't store their reservations in memory - throw new UnsupportedOperationException(); - } - private class TestFateTxStore implements FateTxStore { private final FateId fateId; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index e8551d1dfe0..7c5fac90e98 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import java.io.BufferedWriter; import java.io.File; @@ -788,9 +787,8 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore mfs = - new MetaFateStore<>(fateZkPath, zk, context.getZooCache(), createDummyLockID()); - UserFateStore ufs = new UserFateStore<>(context, createDummyLockID()); + MetaFateStore mfs = new MetaFateStore<>(fateZkPath, zk, null, null); + UserFateStore ufs = new UserFateStore<>(context, null, null); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); Map> readOnlyFateStores = diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index 3fff1301d7a..161c1012836 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -84,6 +84,7 @@ import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLock.LockLossReason; @@ -1149,11 +1150,13 @@ boolean canSuspendTablets() { } try { + Predicate isLockHeld = + lock -> ServiceLock.isLockHeld(context.getZooCache(), lock); var metaInstance = initializeFateInstance(context, new MetaFateStore<>(getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter(), context.getZooCache(), managerLock.getLockID())); - var userInstance = initializeFateInstance(context, - new UserFateStore<>(context, AccumuloTable.FATE.tableName(), managerLock.getLockID())); + context.getZooReaderWriter(), managerLock.getLockID(), isLockHeld)); + var userInstance = initializeFateInstance(context, new UserFateStore<>(context, + AccumuloTable.FATE.tableName(), managerLock.getLockID(), isLockHeld)); if (!fateRefs.compareAndSet(null, Map.of(FateInstanceType.META, metaInstance, FateInstanceType.USER, userInstance))) { @@ -1265,7 +1268,6 @@ protected Fate initializeFateInstance(ServerContext context, FateStore< final Fate fateInstance = new Fate<>(this, store, TraceRepo::toLogString, getConfiguration()); - fateInstance.startDeadReservationCleaner(); var fateCleaner = new FateCleaner<>(store, Duration.ofHours(8), this::getSteadyTime); ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java index 35d5a65a291..3a27a13a8ed 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java @@ -18,8 +18,6 @@ */ package org.apache.accumulo.manager.metrics.fate; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; - import java.util.List; import java.util.Map.Entry; import java.util.concurrent.ScheduledExecutorService; @@ -74,8 +72,7 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter(), - context.getZooCache(), createDummyLockID()); + this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter(), null, null); } catch (KeeperException ex) { throw new IllegalStateException( "FATE Metrics - Failed to create zoo store - metrics unavailable", ex); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java index 3c5f035f943..890249ef26d 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java @@ -19,7 +19,6 @@ package org.apache.accumulo.manager.upgrade; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING; import static org.apache.accumulo.server.AccumuloDataVersion.REMOVE_DEPRECATIONS_FOR_VERSION_3; import static org.apache.accumulo.server.AccumuloDataVersion.ROOT_TABLET_META_CHANGES; @@ -308,9 +307,8 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - final ReadOnlyFateStore fate = - new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter(), context.getZooCache(), createDummyLockID()); + final ReadOnlyFateStore fate = new MetaFateStore<>( + context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter(), null, null); try (var idStream = fate.list()) { if (idStream.findFirst().isPresent()) { throw new AccumuloException("Aborting upgrade because there are" diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index 7bf2740e5ab..b1ca3340d82 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.compaction; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP1; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP2; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP3; @@ -234,7 +233,7 @@ public void testExternalCompaction() throws Exception { public void testCompactionCommitAndDeadDetectionRoot() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), ctx.getZooCache(), createDummyLockID()); + ctx.getZooReaderWriter(), null, null); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var tableId = ctx.getTableId(AccumuloTable.ROOT.tableName()); @@ -253,7 +252,7 @@ public void testCompactionCommitAndDeadDetectionRoot() throws Exception { public void testCompactionCommitAndDeadDetectionMeta() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), ctx.getZooCache(), createDummyLockID()); + ctx.getZooReaderWriter(), null, null); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { // Metadata table by default already has 2 tablets @@ -275,7 +274,7 @@ public void testCompactionCommitAndDeadDetectionUser() throws Exception { final String tableName = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { - UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID()); + UserFateStore userFateStore = new UserFateStore<>(ctx, null, null); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); c.tableOperations().create(tableName, new NewTableConfiguration().withSplits(splits)); @@ -298,10 +297,9 @@ public void testCompactionCommitAndDeadDetectionAll() throws Exception { final String userTable = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { - UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID()); - FateStore metaFateStore = - new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), - ctx.getZooCache(), createDummyLockID()); + UserFateStore userFateStore = new UserFateStore<>(ctx, null, null); + FateStore metaFateStore = new MetaFateStore<>( + ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), null, null); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java index 2bd54477e41..00847ad5f78 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java @@ -492,9 +492,7 @@ protected Fate initializeFate(FateStore store) { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - Fate fate = new Fate<>(new TestEnv(), store, r -> r + "", config); - fate.startDeadReservationCleaner(); - return fate; + return new Fate<>(new TestEnv(), store, r -> r + "", config); } protected abstract TStatus getTxStatus(ServerContext sctx, FateId fateId); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java index 78113394eb3..f448149d76f 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java @@ -660,9 +660,7 @@ private Fate initializeFate(FateStore store) { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - Fate fate = new Fate<>(new TestEnv(), store, Object::toString, config); - fate.startDeadReservationCleaner(); - return fate; + return new Fate<>(new TestEnv(), store, Object::toString, config); } private boolean wordIsTStatus(String word) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index 4a43601ca68..3a70acb7cbc 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -19,9 +19,6 @@ package org.apache.accumulo.test.fate; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -39,6 +36,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.clientImpl.ClientContext; @@ -52,13 +50,11 @@ import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.user.UserFateStore; -import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.test.util.Wait; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; -import org.easymock.EasyMock; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -127,11 +123,11 @@ private void testReserveUnreserve(FateInstanceType storeType) throws Exception { if (isUserStore) { createFateTable(client, tableName); - store1 = new UserFateStore<>(client, tableName, lock1); - store2 = new UserFateStore<>(client, tableName, lock2); + store1 = new UserFateStore<>(client, tableName, lock1, null); + store2 = new UserFateStore<>(client, tableName, lock2, null); } else { - store1 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock1); - store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); + store1 = new MetaFateStore<>(FATE_DIR, zk, lock1, null); + store2 = new MetaFateStore<>(FATE_DIR, zk, lock2, null); } // Create the fate ids using store1 @@ -193,9 +189,9 @@ private void testReserveNonExistentTxn(FateInstanceType storeType) throws Except if (isUserStore) { createFateTable(client, tableName); - store = new UserFateStore<>(client, tableName, lock); + store = new UserFateStore<>(client, tableName, lock, null); } else { - store = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock); + store = new MetaFateStore<>(FATE_DIR, zk, lock, null); } assertThrows(IllegalStateException.class, () -> store.reserve(fakeFateId)); @@ -219,9 +215,9 @@ private void testReserveReservedAndUnreserveUnreserved(FateInstanceType storeTyp if (isUserStore) { createFateTable(client, tableName); - store = new UserFateStore<>(client, tableName, lock); + store = new UserFateStore<>(client, tableName, lock, null); } else { - store = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock); + store = new MetaFateStore<>(FATE_DIR, zk, lock, null); } // Create some FateIds and ensure that they can be reserved @@ -267,9 +263,9 @@ private void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType st if (isUserStore) { createFateTable(client, tableName); - store = new UserFateStore<>(client, tableName, lock); + store = new UserFateStore<>(client, tableName, lock, null); } else { - store = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock); + store = new MetaFateStore<>(FATE_DIR, zk, lock, null); } // Create some FateIds and ensure that they can be reserved @@ -321,22 +317,24 @@ private void testMultipleFateInstances(FateInstanceType storeType) throws Except final SleepingTestEnv testEnv2 = new SleepingTestEnv(50); final ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); final ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + final Set liveLocks = new HashSet<>(); + final Predicate isLockHeld = liveLocks::contains; if (isUserStore) { createFateTable(client, tableName); - store1 = new UserFateStore<>(client, tableName, lock1); - store2 = new UserFateStore<>(client, tableName, lock2); + store1 = new UserFateStore<>(client, tableName, lock1, isLockHeld); + store2 = new UserFateStore<>(client, tableName, lock2, isLockHeld); } else { - store1 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock1); - store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); + store1 = new MetaFateStore<>(FATE_DIR, zk, lock1, isLockHeld); + store2 = new MetaFateStore<>(FATE_DIR, zk, lock2, isLockHeld); } + liveLocks.add(lock1); + liveLocks.add(lock2); Fate fate1 = new Fate<>(testEnv1, store1, Object::toString, DefaultConfiguration.getInstance()); - fate1.startDeadReservationCleaner(); Fate fate2 = new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); - fate2.startDeadReservationCleaner(); for (int i = 0; i < numFateIds; i++) { FateId fateId; @@ -382,38 +380,28 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce Integer.parseInt(Property.MANAGER_FATE_THREADPOOL_SIZE.getDefaultValue()); final boolean isUserStore = storeType.equals(FateInstanceType.USER); final Set allIds = new HashSet<>(); - final FateStore mockedStore1, store2; + final FateStore store1, store2; final LatchTestEnv testEnv1 = new LatchTestEnv(); final LatchTestEnv testEnv2 = new LatchTestEnv(); final ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); final ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + final Set liveLocks = new HashSet<>(); + final Predicate isLockHeld = liveLocks::contains; Map reservations; if (isUserStore) { createFateTable(client, tableName); - mockedStore1 = EasyMock.createMockBuilder(UserFateStore.class) - .withConstructor(ClientContext.class, String.class, ZooUtil.LockID.class) - .withArgs(client, tableName, lock1).addMockedMethod("isDeadReservation").createMock(); + store1 = new UserFateStore<>(client, tableName, lock1, isLockHeld); } else { - mockedStore1 = EasyMock.createMockBuilder(MetaFateStore.class) - .withConstructor(String.class, ZooReaderWriter.class, ZooCache.class, - ZooUtil.LockID.class) - .withArgs(FATE_DIR, zk, client.getZooCache(), lock1).addMockedMethod("isDeadReservation") - .createMock(); - } - // Define isDeadReservation() for mockedStore1 as always being false. We don't want - // fate1/mockedStore1 to delete any reservations yet (we are simulating that the - // Manager is alive right now) - expect(mockedStore1.isDeadReservation(anyObject(FateStore.FateReservation.class))) - .andReturn(false).anyTimes(); - replay(mockedStore1); + store1 = new MetaFateStore<>(FATE_DIR, zk, lock1, isLockHeld); + } + liveLocks.add(lock1); Fate fate1 = - new Fate<>(testEnv1, mockedStore1, Object::toString, DefaultConfiguration.getInstance()); - fate1.startDeadReservationCleaner(); + new Fate<>(testEnv1, store1, Object::toString, DefaultConfiguration.getInstance()); // Ensure nothing is reserved yet - assertTrue(mockedStore1.getActiveReservations().isEmpty()); + assertTrue(store1.getActiveReservations().isEmpty()); // Create transactions for (int i = 0; i < numFateIds; i++) { @@ -428,50 +416,41 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce Wait.waitFor(() -> testEnv1.numWorkers.get() == numFateIds); // Each fate worker will be hung up working (IN_PROGRESS) on a single transaction - // Verify mockedStore1 has the transactions reserved and that they were reserved with lock1 - reservations = mockedStore1.getActiveReservations(); + // Verify store1 has the transactions reserved and that they were reserved with lock1 + reservations = store1.getActiveReservations(); assertEquals(allIds, reservations.keySet()); reservations.values().forEach( res -> assertTrue(FateStore.FateReservation.locksAreEqual(lock1, res.getLockID()))); if (isUserStore) { - store2 = new UserFateStore<>(client, tableName, lock2); + store2 = new UserFateStore<>(client, tableName, lock2, isLockHeld); } else { - store2 = new MetaFateStore<>(FATE_DIR, zk, client.getZooCache(), lock2); + store2 = new MetaFateStore<>(FATE_DIR, zk, lock2, isLockHeld); } // Verify store2 can see the reserved transactions even though they were reserved using - // mockedStore1 + // store1 reservations = store2.getActiveReservations(); assertEquals(allIds, reservations.keySet()); reservations.values().forEach( res -> assertTrue(FateStore.FateReservation.locksAreEqual(lock1, res.getLockID()))); // Simulate what would happen if the Manager using the Fate object (fate1) died. - // ServerLock.isLockHeld(...) would return false for the LockId of the Manager that died - // (in this case, lock1). - - // Redefine what is considered "dead" as those whose locks match lock1 - EasyMock.reset(mockedStore1); - expect(mockedStore1.isDeadReservation(anyObject(FateStore.FateReservation.class))) - .andAnswer(() -> { - FateStore.FateReservation reservation = - (FateStore.FateReservation) EasyMock.getCurrentArguments()[0]; - return FateStore.FateReservation.locksAreEqual(reservation.getLockID(), lock1); - }).anyTimes(); - replay(mockedStore1); + // isLockHeld would return false for the LockId of the Manager that died (in this case, lock1) + // and true for the new Manager's lock (lock2) + liveLocks.remove(lock1); + liveLocks.add(lock2); // Create the new Fate/start the Fate threads (the work finder and the workers). // The DeadReservationCleaner for fate2 should not run/have no effect since we // already have a DeadReservationCleaner for storeType running from fate1. Fate fate2 = new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); - fate2.startDeadReservationCleaner(); // Wait for the "dead" reservations to be deleted and picked up again (reserved using // fate2/store2/lock2 now). // They are considered "dead" if they are held by lock1 in this test. We don't have to worry - // about fate1/mockedStore1/lock1 being used to reserve the transactions again since all + // about fate1/store1/lock1 being used to reserve the transactions again since all // the workers for fate1 are hung up Wait.waitFor(() -> { Map store2Reservations = store2.getActiveReservations(); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java index 37aecd358dd..f8a3621d130 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.fate.meta; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -76,8 +75,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, createDummyLockID(), - maxDeferred, fateIdGenerator), sctx); + testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, null, maxDeferred, + fateIdGenerator), sctx); } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java index 64dedafed3f..5df36693dab 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java @@ -39,8 +39,6 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred String path = ZK_ROOT + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); zk.mkdirs(ZK_ROOT); - testMethod.execute( - new MetaFateStore<>(path, zk, sctx.getZooCache(), AbstractFateStore.createDummyLockID()), - sctx); + testMethod.execute(new MetaFateStore<>(path, zk, null, null), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java index 84cfbb7d1f1..13508054672 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java @@ -18,8 +18,6 @@ */ package org.apache.accumulo.test.fate.meta; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; - import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.MetaFateStore; @@ -34,7 +32,6 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, ServerContext sctx = getCluster().getServerContext(); String path = sctx.getZooKeeperRoot() + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); - testMethod.execute(new MetaFateStore<>(path, zk, sctx.getZooCache(), createDummyLockID()), - sctx); + testMethod.execute(new MetaFateStore<>(path, zk, null, null), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java index 9ad75f84911..423009c7938 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.fate.meta; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -78,8 +77,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, - createDummyLockID(), maxDeferred, fateIdGenerator); + MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, null, + maxDeferred, fateIdGenerator); // Check that the store has no transactions before and after each test assertEquals(0, store.list().count()); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java index 0a1e6ae4d03..5584567cd59 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java @@ -195,13 +195,6 @@ public void testReservations() throws Exception { status = new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); assertEquals(REJECTED, status); - status = - new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); - assertEquals(REJECTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); - assertEquals(REJECTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); - assertEquals(REJECTED, status); // Initialize the column and ensure we can't do it twice status = new FateMutatorImpl<>(context, table, fateId).putInitReserveColVal().tryMutate(); @@ -213,30 +206,9 @@ public void testReservations() throws Exception { status = new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); assertEquals(REJECTED, status); - status = - new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); - assertEquals(REJECTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); - assertEquals(REJECTED, status); - // It is considered unreserved since it has been initialized - status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); - assertEquals(ACCEPTED, status); - - // Should be able to reserve status = new FateMutatorImpl<>(context, table, fateId).putReservedTx(reservation).tryMutate(); assertEquals(ACCEPTED, status); - // Ensure that it is reserved - status = - new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); - assertEquals(ACCEPTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireReserved(wrongReservation) - .tryMutate(); - assertEquals(REJECTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); - assertEquals(ACCEPTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); - assertEquals(REJECTED, status); // Should not be able to reserve when it is already reserved status = new FateMutatorImpl<>(context, table, fateId).putReservedTx(wrongReservation).tryMutate(); @@ -254,15 +226,6 @@ public void testReservations() throws Exception { status = new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); assertEquals(REJECTED, status); - - // Ensure that it is unreserved - status = - new FateMutatorImpl<>(context, table, fateId).requireReserved(reservation).tryMutate(); - assertEquals(REJECTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireReserved().tryMutate(); - assertEquals(REJECTED, status); - status = new FateMutatorImpl<>(context, table, fateId).requireUnreserved().tryMutate(); - assertEquals(ACCEPTED, status); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java index 49d310cc1e5..36c1cc20bb2 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.fate.user; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; import java.util.stream.StreamSupport; @@ -62,7 +61,7 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); testMethod.execute( - new UserFateStore<>(client, table, createDummyLockID(), maxDeferred, fateIdGenerator), + new UserFateStore<>(client, table, null, null, maxDeferred, fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java index c5fa134cc66..7706604deed 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java @@ -34,8 +34,9 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute(new UserFateStore<>(client, table, AbstractFateStore.createDummyLockID(), - maxDeferred, fateIdGenerator), getCluster().getServerContext()); + testMethod.execute( + new UserFateStore<>(client, table, null, null, maxDeferred, fateIdGenerator), + getCluster().getServerContext()); client.tableOperations().delete(table); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java index db3341ecfa3..e203da4f7c7 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java @@ -18,8 +18,6 @@ */ package org.apache.accumulo.test.fate.user; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; - import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.test.fate.FateOpsCommandsIT; @@ -28,7 +26,7 @@ public class UserFateOpsCommandsIT extends FateOpsCommandsIT { @Override public void executeTest(FateTestExecutor testMethod, int maxDeferred, AbstractFateStore.FateIdGenerator fateIdGenerator) throws Exception { - testMethod.execute(new UserFateStore<>(getCluster().getServerContext(), createDummyLockID()), + testMethod.execute(new UserFateStore<>(getCluster().getServerContext(), null, null), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java index 8cb66cb43b8..1dedd4b6278 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.fate.user; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.fate.user.UserFateStore.getRowId; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; @@ -58,7 +57,7 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); testMethod.execute( - new UserFateStore<>(client, table, createDummyLockID(), maxDeferred, fateIdGenerator), + new UserFateStore<>(client, table, null, null, maxDeferred, fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java index 7efbe676b2a..979038f7b82 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.fate.user; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -87,7 +86,7 @@ private static class TestUserFateStore extends UserFateStore { // use the list of fateIds to simulate collisions on fateIds public TestUserFateStore(ClientContext context, String tableName, List fateIds) { - super(context, tableName, createDummyLockID()); + super(context, tableName, null, null); this.fateIdIterator = fateIds.iterator(); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index 7153d2a1248..13567829c16 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -21,7 +21,6 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -264,11 +263,10 @@ public void getFateStatus() { InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); MetaFateStore mfs = - new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, - context.getZooCache(), createDummyLockID()); + new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, null, null); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); - UserFateStore ufs = new UserFateStore<>(context, createDummyLockID()); + UserFateStore ufs = new UserFateStore<>(context, null, null); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); @@ -358,8 +356,8 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - MetaFateStore mfs = new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, - zk, context.getZooCache(), createDummyLockID()); + MetaFateStore mfs = + new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, null, null); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); AdminUtil.FateStatus fateStatus = admin.getStatus(mfs, zk, lockPath, null, null, null); @@ -389,7 +387,7 @@ private boolean lookupFateInAccumulo(final String tableName) throws KeeperExcept log.trace("tid: {}", tableId); - UserFateStore ufs = new UserFateStore<>(context, createDummyLockID()); + UserFateStore ufs = new UserFateStore<>(context, null, null); AdminUtil.FateStatus fateStatus = admin.getStatus(ufs, null, null, null); log.trace("current fates: {}", fateStatus.getTransactions().size()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java index bb3d29fc6bf..720b4862ef9 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test.functional; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FLUSH_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -232,8 +231,8 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore mfs = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, - zk, context.getZooCache(), createDummyLockID()); + MetaFateStore mfs = + new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, null, null); var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); return admin.getStatus(mfs, zk, lockPath, null, null, null); } catch (KeeperException | InterruptedException e) { From 176b9c34f7702efb67773a93d8589d826d13682d Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 2 Jul 2024 10:11:39 -0400 Subject: [PATCH 09/17] Build fix --- .../src/main/java/org/apache/accumulo/server/util/Admin.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index 3fdd345e756..00ad73a9596 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -1000,9 +1000,9 @@ private static long printDanglingFateOperations(ServerContext context, } }; - UserFateStore ufs = new UserFateStore<>(context); + UserFateStore ufs = new UserFateStore<>(context, null, null); MetaFateStore mfs = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter()); + context.getZooReaderWriter(), null, null); LoadingCache fateStatusCache = Caffeine.newBuilder() .maximumSize(100_000).expireAfterWrite(10, TimeUnit.SECONDS).build(fateId -> { if (fateId.getType() == FateInstanceType.META) { From 6760035608387ae537af7050106c9645aac19aae Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 23 Jul 2024 14:48:29 -0400 Subject: [PATCH 10/17] Changes: - Stricter check of the column value for fate reservations. If anything unexpected is seen, an error is now thrown. - Simplified MetaFateStore.getActiveReservations() to only read from ZooKeeper once. - Combined the two scans that were done in AbstractFateStore.runnable() into one. This meant adding FateReservation to FateIdStatus and refactoring Meta/UserFateStore.getTransactions(). - No longer use/store a string representation of the FateReservation (was used in UserFateStore). Now, only the serialized value is used. This keeps the usage of FateReservation consistent across Meta and UserFateStore. It was also unneccessary, so simplifies code. - Moved AbstractFateStore.createAndReserve() implementation into Meta and UserFateStore, and rewrote the impl for each to work with the new way reservations are stored. This also allowed me to delete methods that were only used by AFS.createAndReserve(): create(FateId, FateKey), getStatusAndKey(FateId), create(FateKey). - Fixed how concurrentStatusChangeCallers was decremented in AbstractFateStore.waitForStatusChange() - Small change to MetaFateStore.deleteDeadReservations() to avoid reading from ZK unnecessarily - Added isReservedBy() method to MetaFateStore.NodeValue to avoid code duplication and make the code more clear. - Since the FateIdStatus now has the FateReservation, realized Meta and UserFateStore.getActiveReservations() could now be simplified to just call list(). This also made the impls the same, so moved to AbstractFateStore. This also made me realize that I had put getActiveReservations() method signature in FateStore, but would be better suited for ReadOnlyFateStore, so moved it there. - Deleted FateStore.isReserved(FateId)... No longer needed/used - Moved UNKNOWN status check in AbstractFateStore.reserve() into waiting loop - Now log when a dead reservation is detected and deleted - Minor change to Fate: no longer create the executor for the dead reservation cleaner if it's not going to be used --- .../accumulo/core/fate/AbstractFateStore.java | 173 ++++--------- .../org/apache/accumulo/core/fate/Fate.java | 35 ++- .../apache/accumulo/core/fate/FateKey.java | 3 + .../apache/accumulo/core/fate/FateStore.java | 76 ++---- .../accumulo/core/fate/MetaFateStore.java | 118 +++++---- .../accumulo/core/fate/ReadOnlyFateStore.java | 13 +- .../core/fate/WrappedFateTxStore.java | 6 - .../accumulo/core/fate/user/FateMutator.java | 2 +- .../core/fate/user/FateMutatorImpl.java | 10 +- .../core/fate/user/FateStatusFilter.java | 12 +- .../core/fate/user/UserFateStore.java | 239 ++++++++++-------- .../accumulo/core/logging/FateLogger.java | 5 - .../apache/accumulo/core/fate/TestStore.java | 33 ++- .../accumulo/test/fate/FateOpsCommandsIT.java | 7 + .../accumulo/test/fate/FateStoreIT.java | 3 +- .../test/fate/user/FateMutatorImplIT.java | 4 +- 16 files changed, 358 insertions(+), 381 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index a3cbf829605..ab90dbd2269 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.fate; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.ALL_STATUSES; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -41,11 +42,11 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Predicate; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.time.NanoTime; import org.slf4j.Logger; @@ -79,9 +80,9 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { protected final ZooUtil.LockID lockID; protected final Predicate isLockHeld; protected final Map deferred; + protected final FateIdGenerator fateIdGenerator; private final int maxDeferred; private final AtomicBoolean deferredOverflow = new AtomicBoolean(); - private final FateIdGenerator fateIdGenerator; // This is incremented each time a transaction is unreserved that was runnable private final SignalCount unreservedRunnableCount = new SignalCount(); @@ -133,13 +134,13 @@ public static Object deserialize(byte[] ser) { @Override public FateTxStore reserve(FateId fateId) { - Preconditions.checkState(!_getStatus(fateId).equals(TStatus.UNKNOWN), - "Attempted to reserve a tx that does not exist: " + fateId); var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25)) .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5) .logInterval(Duration.ofMinutes(3)).createRetry(); Optional> reserveAttempt = tryReserve(fateId); while (reserveAttempt.isEmpty()) { + Preconditions.checkState(!_getStatus(fateId).equals(TStatus.UNKNOWN), + "Attempted to reserve a tx that does not exist: " + fateId); try { retry.waitForNextAttempt(log, "Attempting to reserve " + fateId); } catch (InterruptedException e) { @@ -172,7 +173,8 @@ public void runnable(AtomicBoolean keepWaiting, Consumer idConsumer) { // first var transactions = Stream.concat(inProgress, other); transactions.filter(fateIdStatus -> isRunnable(fateIdStatus.getStatus())) - .map(FateIdStatus::getFateId).filter(fateId -> { + .filter(fateIdStatus -> { + var fateId = fateIdStatus.getFateId(); var deferredTime = deferred.get(fateId); if (deferredTime != null) { if (deferredTime.elapsed().isNegative()) { @@ -182,10 +184,10 @@ public void runnable(AtomicBoolean keepWaiting, Consumer idConsumer) { deferred.remove(fateId); } } - return !isReserved(fateId); - }).forEach(fateId -> { + return fateIdStatus.getFateReservation().isEmpty(); + }).forEach(fateIdStatus -> { seen.incrementAndGet(); - idConsumer.accept(fateId); + idConsumer.accept(fateIdStatus.getFateId()); }); } @@ -236,6 +238,12 @@ public ReadOnlyFateTxStore read(FateId fateId) { return newUnreservedFateTxStore(fateId); } + @Override + public Map getActiveReservations() { + return list().filter(entry -> entry.getFateReservation().isPresent()).collect(Collectors + .toMap(FateIdStatus::getFateId, entry -> entry.getFateReservation().orElseThrow())); + } + protected boolean isRunnable(TStatus status) { return status == TStatus.IN_PROGRESS || status == TStatus.FAILED_IN_PROGRESS || status == TStatus.SUBMITTED; @@ -267,89 +275,13 @@ public int getDeferredCount() { return deferred.size(); } - private Optional create(FateKey fateKey) { - FateId fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); - - try { - create(fateId, fateKey); - } catch (IllegalStateException e) { - Pair> statusAndKey = getStatusAndKey(fateId); - TStatus status = statusAndKey.getFirst(); - Optional tFateKey = statusAndKey.getSecond(); - - // Case 1: Status is NEW so this is unseeded, we can return and allow the calling code - // to reserve/seed as long as the existing key is the same and not different as that would - // mean a collision - if (status == TStatus.NEW) { - Preconditions.checkState(tFateKey.isPresent(), "Tx Key is missing from tid %s", - fateId.getTxUUIDStr()); - Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), - "Collision detected for tid %s", fateId.getTxUUIDStr()); - // Case 2: Status is some other state which means already in progress - // so we can just log and return empty optional - } else { - log.trace("Existing transaction {} already exists for key {} with status {}", fateId, - fateKey, status); - return Optional.empty(); - } - } - - return Optional.of(fateId); + protected void verifyFateKey(FateId fateId, Optional fateKeySeen, + FateKey fateKeyExpected) { + Preconditions.checkState(fateKeySeen.isPresent(), "fate key is missing from fate id " + fateId); + Preconditions.checkState(fateKeySeen.orElseThrow().equals(fateKeyExpected), + "Collision detected for fate id " + fateId); } - @Override - public Optional> createAndReserve(FateKey fateKey) { - // TODO 4131 not confident about this new implementation of createAndReserve. - // Previously, you could reserve before creation, but with the new impl of reservations - // being stored in ZK (MetaFateStore) and the Accumulo Fate table (UserFateStore), creation - // is needed before reservation. - // TODO 4131 the comments in this method also need to be updated. - // Will wait until after review for this method - FateId fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); - final Optional> txStore; - - // First make sure we can reserve in memory the fateId, if not - // we can return an empty Optional as it is reserved and in progress - // This reverses the usual order of creation and then reservation but - // this prevents a race condition by ensuring we can reserve first. - // This will create the FateTxStore before creation but this object - // is not exposed until after creation is finished so there should not - // be any errors. - - // If present we were able to reserve so try and create - if (!isReserved(fateId)) { - try { - var fateIdFromCreate = create(fateKey); - if (fateIdFromCreate.isPresent()) { - Preconditions.checkState(fateId.equals(fateIdFromCreate.orElseThrow()), - "Transaction creation returned unexpected %s, expected %s", fateIdFromCreate, fateId); - txStore = tryReserve(fateId); - } else { - // We already exist in a non-new state then un-reserve and an empty - // Optional will be returned. This is expected to happen when the - // system is busy and operations are not running, and we keep seeding them - txStore = Optional.empty(); - } - } catch (Exception e) { - if (e instanceof IllegalStateException) { - throw e; - } else { - throw new IllegalStateException(e); - } - } - } else { - // Could not reserve so return empty - log.trace("Another thread currently has transaction {} key {} reserved", fateId, fateKey); - txStore = Optional.empty(); - } - - return txStore; - } - - protected abstract void create(FateId fateId, FateKey fateKey); - - protected abstract Pair> getStatusAndKey(FateId fateId); - protected abstract Stream getTransactions(Set statuses); protected abstract TStatus _getStatus(FateId fateId); @@ -388,35 +320,38 @@ public TStatus waitForStatusChange(EnumSet expected) { verifyReserved(false); int currNumCallers = concurrentStatusChangeCallers.incrementAndGet(); - // TODO 4131 - // TODO make the max time a function of the number of concurrent callers, as the number of - // concurrent callers increases then increase the max wait time - // TODO could support signaling within this instance for known events - // TODO made the maxWait low so this would be responsive... that may put a lot of load in the - // case there are lots of things waiting... - // Made maxWait = num of curr callers - var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25)) - .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(currNumCallers)) - .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry(); - - while (true) { - - TStatus status = _getStatus(fateId); - if (expected.contains(status)) { - retry.logCompletion(log, "Waiting on status change for " + fateId + " expected:" - + expected + " status:" + status); - concurrentStatusChangeCallers.decrementAndGet(); - return status; - } - try { - retry.waitForNextAttempt(log, "Waiting on status change for " + fateId + " expected:" - + expected + " status:" + status); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - concurrentStatusChangeCallers.decrementAndGet(); - throw new IllegalStateException(e); + try { + // TODO 4131 + // TODO make the max time a function of the number of concurrent callers, as the number of + // concurrent callers increases then increase the max wait time + // TODO could support signaling within this instance for known events + // TODO made the maxWait low so this would be responsive... that may put a lot of load in + // the case there are lots of things waiting... + // Made maxWait = num of curr callers + var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25)) + .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(currNumCallers)) + .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry(); + + while (true) { + + TStatus status = _getStatus(fateId); + if (expected.contains(status)) { + retry.logCompletion(log, "Waiting on status change for " + fateId + " expected:" + + expected + " status:" + status); + return status; + } + + try { + retry.waitForNextAttempt(log, "Waiting on status change for " + fateId + " expected:" + + expected + " status:" + status); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } } + } finally { + concurrentStatusChangeCallers.decrementAndGet(); } } @@ -475,12 +410,6 @@ public Optional getKey() { return AbstractFateStore.this.getKey(fateId); } - @Override - public Pair> getStatusAndKey() { - verifyReserved(false); - return AbstractFateStore.this.getStatusAndKey(fateId); - } - @Override public FateId getID() { return fateId; diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index fa68825267d..5ed9b090165 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -387,16 +387,19 @@ public Fate(T environment, FateStore store, Function,String> toLogStr // reservation cleaner is not already running for the given store type. // TODO 4131 periodic cleanup runs every 30 seconds // Should this be longer? Shorter? A configurable Property? A function of something? - ScheduledExecutorService deadResCleanerExecutor = ThreadPools.getServerThreadPools() - .createScheduledExecutorService(1, store.type() + "-dead-reservation-cleaner-pool"); - if ((store.type() == FateInstanceType.USER && !userDeadReservationCleanerRunning) - || (store.type() == FateInstanceType.META && !metaDeadReservationCleanerRunning)) { + ScheduledExecutorService deadResCleanerExecutor = null; + boolean isUserStore = store.type() == FateInstanceType.USER; + boolean isMetaStore = store.type() == FateInstanceType.META; + if ((isUserStore && !userDeadReservationCleanerRunning) + || (isMetaStore && !metaDeadReservationCleanerRunning)) { + deadResCleanerExecutor = ThreadPools.getServerThreadPools().createScheduledExecutorService(1, + store.type() + "-dead-reservation-cleaner-pool"); ScheduledFuture deadReservationCleaner = deadResCleanerExecutor .scheduleWithFixedDelay(new DeadReservationCleaner(), 3, 30, SECONDS); ThreadPools.watchCriticalScheduledTask(deadReservationCleaner); - if (store.type() == FateInstanceType.USER) { + if (isUserStore) { userDeadReservationCleanerRunning = true; - } else if (store.type() == FateInstanceType.META) { + } else { metaDeadReservationCleanerRunning = true; } } @@ -572,21 +575,25 @@ public void shutdown(long timeout, TimeUnit timeUnit) { fatePoolWatcher.shutdown(); transactionExecutor.shutdown(); workFinder.interrupt(); - deadResCleanerExecutor.shutdown(); + if (deadResCleanerExecutor != null) { + deadResCleanerExecutor.shutdown(); + } } if (timeout > 0) { long start = System.nanoTime(); - while ((System.nanoTime() - start) < timeUnit.toNanos(timeout) && (workFinder.isAlive() - || !transactionExecutor.isTerminated() || !deadResCleanerExecutor.isTerminated())) { + while ((System.nanoTime() - start) < timeUnit.toNanos(timeout) + && (workFinder.isAlive() || !transactionExecutor.isTerminated() + || (deadResCleanerExecutor != null && !deadResCleanerExecutor.isTerminated()))) { try { if (!transactionExecutor.awaitTermination(1, SECONDS)) { log.debug("Fate {} is waiting for worker threads to terminate", store.type()); continue; } - if (!deadResCleanerExecutor.awaitTermination(1, SECONDS)) { + if (deadResCleanerExecutor != null + && !deadResCleanerExecutor.awaitTermination(1, SECONDS)) { log.debug("Fate {} is waiting for dead reservation cleaner thread to terminate", store.type()); continue; @@ -603,18 +610,20 @@ public void shutdown(long timeout, TimeUnit timeUnit) { } if (workFinder.isAlive() || !transactionExecutor.isTerminated() - || !deadResCleanerExecutor.isTerminated()) { + || (deadResCleanerExecutor != null && !deadResCleanerExecutor.isTerminated())) { log.warn( "Waited for {}ms for all fate {} background threads to stop, but some are still running. workFinder:{} transactionExecutor:{} deadResCleanerExecutor:{}", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), store.type(), workFinder.isAlive(), !transactionExecutor.isTerminated(), - !deadResCleanerExecutor.isTerminated()); + (deadResCleanerExecutor != null && !deadResCleanerExecutor.isTerminated())); } } // interrupt the background threads transactionExecutor.shutdownNow(); - deadResCleanerExecutor.shutdownNow(); + if (deadResCleanerExecutor != null) { + deadResCleanerExecutor.shutdownNow(); + } // Update that USER/META dead reservation cleaner is no longer running if (store.type() == FateInstanceType.USER && userDeadReservationCleanerRunning) { diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java b/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java index 6c1663627c7..210621b24c5 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java @@ -32,6 +32,9 @@ public class FateKey { + // TODO 4131 noticed FateKey is logged, but doesn't have a toString() + // a toString() method should be added. + private final FateKeyType type; private final Optional keyExtent; private final Optional compactionId; diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 5640dae1f7e..38dda29e645 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -24,12 +24,12 @@ import java.io.Serializable; import java.io.UncheckedIOException; import java.time.Duration; -import java.util.Map; +import java.util.Arrays; import java.util.Objects; import java.util.Optional; import java.util.UUID; -import java.util.regex.Pattern; +import org.apache.accumulo.core.fate.user.FateMutatorImpl; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.hadoop.io.DataInputBuffer; @@ -136,9 +136,6 @@ class FateReservation { // expose a FateTxStore). private final UUID reservationUUID; private final byte[] serialized; - private static final Pattern UUID_PATTERN = - Pattern.compile("^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"); - private static final Pattern LOCKID_PATTERN = Pattern.compile("^.+/.+\\$[0-9a-fA-F]+$"); private FateReservation(ZooUtil.LockID lockID, UUID reservationUUID) { this.lockID = Objects.requireNonNull(lockID); @@ -150,43 +147,17 @@ public static FateReservation from(ZooUtil.LockID lockID, UUID reservationUUID) return new FateReservation(lockID, reservationUUID); } - public static FateReservation from(byte[] serialized) { - try (DataInputBuffer buffer = new DataInputBuffer()) { - buffer.reset(serialized, serialized.length); - ZooUtil.LockID lockID = new ZooUtil.LockID("", buffer.readUTF()); - UUID reservationUUID = UUID.fromString(buffer.readUTF()); - return new FateReservation(lockID, reservationUUID); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - public static FateReservation from(String fateReservationStr) { - if (isFateReservation(fateReservationStr)) { - String[] fields = fateReservationStr.split(":"); - ZooUtil.LockID lockId = new ZooUtil.LockID("", fields[0]); - UUID reservationUUID = UUID.fromString(fields[1]); - return new FateReservation(lockId, reservationUUID); - } else { - throw new IllegalArgumentException( - "Tried to create a FateReservation from an invalid string: " + fateReservationStr); - } - } - /** - * - * @param fateReservationStr the string from a call to {@link FateReservation#toString()} - * @return true if the string represents a valid FateReservation object, false otherwise + * @param serializedFateRes the value present in the table for the reservation column + * @return true if the array represents a valid serialized FateReservation object, false if it + * represents an unreserved value, error otherwise */ - public static boolean isFateReservation(String fateReservationStr) { - if (fateReservationStr != null) { - String[] fields = fateReservationStr.split(":"); - if (fields.length == 2) { - return LOCKID_PATTERN.matcher(fields[0]).matches() - && UUID_PATTERN.matcher(fields[1]).matches(); - } + public static boolean isFateReservation(byte[] serializedFateRes) { + if (Arrays.equals(serializedFateRes, FateMutatorImpl.NOT_RESERVED)) { + return false; } - return false; + deserialize(serializedFateRes); + return true; } public ZooUtil.LockID getLockID() { @@ -214,7 +185,14 @@ private static byte[] serialize(ZooUtil.LockID lockID, UUID reservationUUID) { } public static FateReservation deserialize(byte[] serialized) { - return FateReservation.from(serialized); + try (DataInputBuffer buffer = new DataInputBuffer()) { + buffer.reset(serialized, serialized.length); + ZooUtil.LockID lockID = new ZooUtil.LockID("", buffer.readUTF()); + UUID reservationUUID = UUID.fromString(buffer.readUTF()); + return new FateReservation(lockID, reservationUUID); + } catch (IOException e) { + throw new UncheckedIOException(e); + } } public static boolean locksAreEqual(ZooUtil.LockID lockID1, ZooUtil.LockID lockID2) { @@ -228,10 +206,12 @@ public String toString() { @Override public boolean equals(Object obj) { + if (obj == this) { + return true; + } if (obj instanceof FateReservation) { FateReservation other = (FateReservation) obj; - return this.lockID.serialize("/").equals(other.lockID.serialize("/")) - && this.reservationUUID.equals(other.reservationUUID); + return Arrays.equals(this.getSerialized(), other.getSerialized()); } return false; } @@ -242,18 +222,6 @@ public int hashCode() { } } - /** - * @param fateId the fateId to check - * @return true if the given fate id is reserved, false otherwise - */ - boolean isReserved(FateId fateId); - - /** - * @return a map of the current active reservations with the keys being the transaction that is - * reserved and the value being the value stored to indicate the transaction is reserved. - */ - Map getActiveReservations(); - /** * Deletes the current reservations which were reserved by a now dead Manager. These reservations * can no longer be worked on so their reservation should be deleted, so they can be picked up and diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index 4cf95419fa5..5d998c25a11 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -29,7 +29,6 @@ import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -110,11 +109,46 @@ public FateId create() { } @Override - protected void create(FateId fateId, FateKey key) { + public Optional> createAndReserve(FateKey fateKey) { + final var reservation = FateReservation.from(lockID, UUID.randomUUID()); + final var fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); + try { - zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, null, key).serialize(), - NodeExistsPolicy.FAIL); - } catch (KeeperException | InterruptedException e) { + byte[] nodeVal = zk.mutateOrCreate(getTXPath(fateId), + new NodeValue(TStatus.NEW, reservation, fateKey).serialize(), currSerNodeVal -> { + // We are only returning a non-null value for the following cases: + // 1) The existing NodeValue for fateId is exactly the same as the value set for the + // node if it doesn't yet exist: + // TStatus = TStatus.NEW, FateReservation = reservation, FateKey = fateKey + // This might occur if there was a ZK server fault and the same write is running a 2nd + // time + // 2) The existing NodeValue for fateId has: + // TStatus = TStatus.NEW, no FateReservation present, FateKey = fateKey + // The fateId is NEW/unseeded and not reserved, so we can allow it to be reserved + NodeValue currNodeVal = new NodeValue(currSerNodeVal); + if (currNodeVal.status == TStatus.NEW && currNodeVal.isReservedBy(reservation)) { + verifyFateKey(fateId, currNodeVal.fateKey, fateKey); + return currSerNodeVal; + } else if (currNodeVal.status == TStatus.NEW && !currNodeVal.isReserved()) { + verifyFateKey(fateId, currNodeVal.fateKey, fateKey); + // NEW/unseeded transaction and not reserved, so we can allow it to be reserved + return new NodeValue(TStatus.NEW, reservation, fateKey).serialize(); + } else { + log.trace( + "fate id {} tstatus {} fate key {} is reserved {} is either currently reserved " + + "or has already been seeded with work (non-NEW status), or both", + fateId, currNodeVal.status, currNodeVal.fateKey.orElse(null), + currNodeVal.isReserved()); + // This will not change the value to null but will return null + return null; + } + }); + if (nodeVal != null) { + return Optional.of(new FateTxStoreImpl(fateId, reservation)); + } else { + return Optional.empty(); + } + } catch (InterruptedException | KeeperException | AcceptableThriftTableOperationException e) { throw new IllegalStateException(e); } } @@ -130,8 +164,7 @@ public Optional> tryReserve(FateId fateId) { // The uuid handles the case where there was a ZK server fault and the write for this thread // went through but that was not acknowledged, and we are reading our own write for 2nd // time. - if (!currNodeVal.isReserved() || (currNodeVal.isReserved() - && currNodeVal.reservation.orElseThrow().equals(reservation))) { + if (!currNodeVal.isReserved() || currNodeVal.isReservedBy(reservation)) { FateKey currFateKey = currNodeVal.fateKey.orElse(null); // Add the FateReservation to the node to reserve return new NodeValue(currNodeVal.status, reservation, currFateKey).serialize(); @@ -150,44 +183,23 @@ public Optional> tryReserve(FateId fateId) { } } - @Override - public boolean isReserved(FateId fateId) { - return getNode(fateId).isReserved(); - } - - @Override - public Map getActiveReservations() { - Map activeReservations = new HashMap<>(); - - try { - for (String strTxId : zk.getChildren(path)) { - String txUUIDStr = strTxId.split("_")[1]; - FateId fateId = FateId.from(fateInstanceType, txUUIDStr); - if (isReserved(fateId)) { - FateReservation reservation = getNode(fateId).reservation.orElseThrow(); - activeReservations.put(fateId, reservation); - } - } - } catch (KeeperException | InterruptedException e) { - throw new RuntimeException(e); - } - - return activeReservations; - } - @Override public void deleteDeadReservations() { for (Map.Entry entry : getActiveReservations().entrySet()) { FateId fateId = entry.getKey(); FateReservation reservation = entry.getValue(); + if (isLockHeld.test(reservation.getLockID())) { + continue; + } try { zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { NodeValue currNodeVal = new NodeValue(currSerNodeVal); // Make sure the current node is still reserved and reserved with the expected reservation // and it is dead - if (currNodeVal.isReserved() && currNodeVal.reservation.orElseThrow().equals(reservation) + if (currNodeVal.isReservedBy(reservation) && !isLockHeld.test(currNodeVal.reservation.orElseThrow().getLockID())) { // Delete the reservation + log.trace("Deleted the dead reservation {} for fate id {}", reservation, fateId); return new NodeValue(currNodeVal.status, null, currNodeVal.fateKey.orElse(null)) .serialize(); } else { @@ -201,12 +213,6 @@ public void deleteDeadReservations() { } } - @Override - protected Pair> getStatusAndKey(FateId fateId) { - final NodeValue node = getNode(fateId); - return new Pair<>(node.status, node.fateKey); - } - @Override public FateInstanceType type() { return fateInstanceType; @@ -319,8 +325,7 @@ public void setStatus(TStatus status) { zk.mutateExisting(getTXPath(fateId), currSerializedData -> { NodeValue currNodeVal = new NodeValue(currSerializedData); // Ensure the FateId is reserved in ZK, and it is reserved with the expected reservation - if (currNodeVal.isReserved() - && currNodeVal.reservation.orElseThrow().equals(this.reservation)) { + if (currNodeVal.isReservedBy(this.reservation)) { FateReservation currFateReservation = currNodeVal.reservation.orElseThrow(); FateKey currFateKey = currNodeVal.fateKey.orElse(null); NodeValue newNodeValue = new NodeValue(status, currFateReservation, currFateKey); @@ -428,17 +433,18 @@ protected void unreserve() { zk.mutateExisting(getTXPath(fateId), currSerNodeVal -> { NodeValue currNodeVal = new NodeValue(currSerNodeVal); FateKey currFateKey = currNodeVal.fateKey.orElse(null); - if (currNodeVal.isReserved() - && currNodeVal.reservation.orElseThrow().equals(this.reservation)) { + if (currNodeVal.isReservedBy(this.reservation)) { // Remove the FateReservation from the NodeValue to unreserve return new NodeValue(currNodeVal.status, null, currFateKey).serialize(); } else { // possible this is running a 2nd time in zk server fault conditions and its first // write went through if (!currNodeVal.isReserved()) { - log.trace("The FATE reservation does not exist in ZK"); + log.trace("The FATE reservation for fate id {} does not exist in ZK", fateId); } else if (!currNodeVal.reservation.orElseThrow().equals(this.reservation)) { - log.debug("The FATE reservation in ZK differs from that in the store"); + log.debug( + "The FATE reservation for fate id {} in ZK differs from that in the store", + fateId); } return null; } @@ -492,13 +498,23 @@ protected Stream getTransactions(Set statuses) { Stream stream = zk.getChildren(path).stream().map(strTxid -> { String txUUIDStr = strTxid.split("_")[1]; FateId fateId = FateId.from(fateInstanceType, txUUIDStr); - // Memoizing for two reasons. First the status may never be requested, so in that case avoid - // the lookup. Second, if its requested multiple times the result will always be consistent. - Supplier statusSupplier = Suppliers.memoize(() -> _getStatus(fateId)); + // Memoizing for two reasons. First the status or reservation may never be requested, so + // in that case avoid the lookup. Second, if it's requested multiple times the result will + // always be consistent. + Supplier>> statusAndResSupplier = + Suppliers.memoize(() -> { + NodeValue zkNode = getNode(fateId); + return new Pair<>(zkNode.status, zkNode.reservation); + }); return new FateIdStatusBase(fateId) { @Override public TStatus getStatus() { - return statusSupplier.get(); + return statusAndResSupplier.get().getFirst(); + } + + @Override + public Optional getFateReservation() { + return statusAndResSupplier.get().getSecond(); } }; }); @@ -590,5 +606,9 @@ byte[] serialize() { public boolean isReserved() { return reservation.isPresent(); } + + public boolean isReservedBy(FateReservation reservation) { + return isReserved() && this.reservation.orElseThrow().equals(reservation); + } } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java index 1fd9cac06be..bbb08f1b4bf 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.EnumSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -29,8 +30,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.accumulo.core.util.Pair; - /** * Read only access to a Transaction Store. * @@ -98,8 +97,6 @@ interface ReadOnlyFateTxStore { Optional getKey(); - Pair> getStatusAndKey(); - /** * Wait for the status of a transaction to change * @@ -134,6 +131,8 @@ interface ReadOnlyFateTxStore { interface FateIdStatus { FateId getFateId(); + Optional getFateReservation(); + TStatus getStatus(); } @@ -156,6 +155,12 @@ interface FateIdStatus { */ Stream list(FateKey.FateKeyType type); + /** + * @return a map of the current active reservations with the keys being the transaction that is + * reserved and the value being the value stored to indicate the transaction is reserved. + */ + Map getActiveReservations(); + /** * Finds all fate ops that are (IN_PROGRESS, SUBMITTED, or FAILED_IN_PROGRESS) and unreserved. Ids * that are found are passed to the consumer. This method will block until at least one runnable diff --git a/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java b/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java index ac5147d4a97..bf0a35721c7 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java @@ -25,7 +25,6 @@ import java.util.Optional; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; -import org.apache.accumulo.core.util.Pair; public class WrappedFateTxStore implements FateStore.FateTxStore { protected final FateStore.FateTxStore wrapped; @@ -64,11 +63,6 @@ public Optional getKey() { return wrapped.getKey(); } - @Override - public Pair> getStatusAndKey() { - return wrapped.getStatusAndKey(); - } - @Override public void setStatus(FateStore.TStatus status) { wrapped.setStatus(status); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java index 416a45ff317..067579f0069 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java @@ -57,7 +57,7 @@ public interface FateMutator { * * @return the FateMutator with this added mutation */ - FateMutator putInitReserveColVal(); + FateMutator putInitReservationVal(); FateMutator putName(byte[] data); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java index c85f530a868..e9a10c1229f 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.core.fate.user; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.core.fate.AbstractFateStore.serialize; import static org.apache.accumulo.core.fate.user.UserFateStore.getRow; import static org.apache.accumulo.core.fate.user.UserFateStore.getRowId; @@ -50,7 +51,7 @@ public class FateMutatorImpl implements FateMutator { - protected static final String NOT_RESERVED = ""; + public static final byte[] NOT_RESERVED = "".getBytes(UTF_8); private final ClientContext context; private final String tableName; @@ -87,21 +88,22 @@ public FateMutator putReservedTx(FateStore.FateReservation reservation) { Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()).setValue(NOT_RESERVED); mutation.addCondition(condition); - TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(reservation.toString())); + TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(reservation.getSerialized())); return this; } @Override public FateMutator putUnreserveTx(FateStore.FateReservation reservation) { Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()).setValue(reservation.toString()); + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()) + .setValue(reservation.getSerialized()); mutation.addCondition(condition); TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(NOT_RESERVED)); return this; } @Override - public FateMutator putInitReserveColVal() { + public FateMutator putInitReservationVal() { Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); mutation.addCondition(condition); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java index e586a646c38..df179e383b8 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java @@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.ScannerBase; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.iterators.Filter; import org.apache.accumulo.core.iterators.IteratorEnvironment; @@ -54,8 +55,15 @@ public void init(SortedKeyValueIterator source, Map op @Override public boolean accept(Key k, Value v) { - var tstatus = ReadOnlyFateStore.TStatus.valueOf(v.toString()); - return valuesToAccept.contains(tstatus); + // We may see TStatus values or FateReservation values with how this filter is used, + // only accept TStatus values, return false on FateReservation values, error otherwise + try { + var tstatus = ReadOnlyFateStore.TStatus.valueOf(v.toString()); + return valuesToAccept.contains(tstatus); + } catch (IllegalArgumentException e) { + FateStore.FateReservation.isFateReservation(v.get()); + return false; + } } public static void configureScanner(ScannerBase scanner, diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index 42f5e5c45f3..6b2325bd32f 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -18,20 +18,22 @@ */ package org.apache.accumulo.core.fate.user; +import java.io.IOException; import java.io.Serializable; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.SortedMap; import java.util.UUID; import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; @@ -50,10 +52,10 @@ import org.apache.accumulo.core.fate.user.schema.FateSchema.TxColumnFamily; import org.apache.accumulo.core.fate.user.schema.FateSchema.TxInfoColumnFamily; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.iterators.user.WholeRowIterator; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.ColumnFQ; -import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.hadoop.io.Text; import org.slf4j.Logger; @@ -105,7 +107,7 @@ public FateId create() { } var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW) - .putCreateTime(System.currentTimeMillis()).putInitReserveColVal().tryMutate(); + .putCreateTime(System.currentTimeMillis()).putInitReservationVal().tryMutate(); switch (status) { case ACCEPTED: @@ -126,35 +128,86 @@ public FateId getFateId() { } @Override - protected void create(FateId fateId, FateKey fateKey) { - final int maxAttempts = 5; - - for (int attempt = 0; attempt < maxAttempts; attempt++) { - - if (attempt >= 1) { - log.debug("Failed to create transaction with fateId {} and fateKey {}, trying again", - fateId, fateKey); - UtilWaitThread.sleep(100); - } - - var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) - .putCreateTime(System.currentTimeMillis()).putInitReserveColVal().tryMutate(); + public Optional> createAndReserve(FateKey fateKey) { + final var reservation = FateReservation.from(lockID, UUID.randomUUID()); + final var fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); + Optional> txStore = Optional.empty(); + + var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) + .putInitReservationVal().putReservedTx(reservation) + .putCreateTime(System.currentTimeMillis()).tryMutate(); + + switch (status) { + case ACCEPTED: + txStore = Optional.of(new FateTxStoreImpl(fateId, reservation)); + break; + case UNKNOWN: + case REJECTED: + // If the status is UNKNOWN, the mutation may or may not have been written. We need to + // check if it was written only returning the FateTxStore if it was. + // If the status is REJECTED, we need to check what about the mutation was REJECTED: + // 1) If there is a collision with existing fate id, throw error + // 2) If the fate id is already reserved, return an empty optional + // 3) If the fate id is still NEW/unseeded and unreserved, we can try to reserve it + try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { + scanner.setRange(getRow(fateId)); + scanner.fetchColumn(TxColumnFamily.STATUS_COLUMN.getColumnFamily(), + TxColumnFamily.STATUS_COLUMN.getColumnQualifier()); + scanner.fetchColumn(TxColumnFamily.TX_KEY_COLUMN.getColumnFamily(), + TxColumnFamily.TX_KEY_COLUMN.getColumnQualifier()); + scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); + TStatus statusSeen = TStatus.UNKNOWN; + Optional fateKeySeen = Optional.empty(); + Optional reservationSeen = Optional.empty(); + + for (Map.Entry entry : scanner.stream().collect(Collectors.toList())) { + Text colf = entry.getKey().getColumnFamily(); + Text colq = entry.getKey().getColumnQualifier(); + Value val = entry.getValue(); + + switch (colq.toString()) { + case TxColumnFamily.STATUS: + statusSeen = TStatus.valueOf(val.toString()); + break; + case TxColumnFamily.TX_KEY: + fateKeySeen = Optional.of(FateKey.deserialize(val.get())); + break; + case TxColumnFamily.RESERVATION: + if (FateReservation.isFateReservation(val.get())) { + reservationSeen = Optional.of(FateReservation.deserialize(val.get())); + } + break; + default: + throw new IllegalStateException("Unexpected column seen: " + colf + ":" + colq); + } + } - switch (status) { - case ACCEPTED: - return; - case UNKNOWN: - continue; - case REJECTED: - throw new IllegalStateException("Attempt to create transaction with fateId " + fateId - + " and fateKey " + fateKey + " was rejected"); - default: - throw new IllegalStateException("Unknown status " + status); - } + // This will be the case if the mutation status is UNKNOWN but the mutation was written + if (statusSeen == TStatus.NEW && reservationSeen.isPresent() + && reservationSeen.orElseThrow().equals(reservation)) { + verifyFateKey(fateId, fateKeySeen, fateKey); + txStore = Optional.of(new FateTxStoreImpl(fateId, reservation)); + } else if (statusSeen == TStatus.NEW && reservationSeen.isEmpty()) { + verifyFateKey(fateId, fateKeySeen, fateKey); + // NEW/unseeded transaction and not reserved, so we can allow it to be reserved + // we tryReserve() since another thread may have reserved it since the scan + txStore = tryReserve(fateId); + } else { + log.trace( + "fate id {} tstatus {} fate key {} is reserved {} is either currently reserved " + + "or has already been seeded with work (non-NEW status), or both", + fateId, statusSeen, fateKeySeen.orElse(null), reservationSeen.isPresent()); + } + } catch (TableNotFoundException e) { + throw new IllegalStateException(tableName + " not found!", e); + } + break; + default: + throw new IllegalStateException("Unknown status " + status); } - throw new IllegalStateException("Failed to create transaction with fateId " + fateId - + " and fateKey " + fateKey + " after " + maxAttempts + " attempts"); + return txStore; } @Override @@ -178,8 +231,8 @@ public Optional> tryReserve(FateId fateId) { scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); FateReservation persistedRes = scanner.stream() - .filter(entry -> FateReservation.isFateReservation(entry.getValue().toString())) - .map(entry -> FateReservation.from(entry.getValue().toString())).findFirst() + .filter(entry -> FateReservation.isFateReservation(entry.getValue().get())) + .map(entry -> FateReservation.deserialize(entry.getValue().get())).findFirst() .orElse(null); if (persistedRes != null && persistedRes.equals(reservation)) { return Optional.of(new FateTxStoreImpl(fateId, reservation)); @@ -191,51 +244,21 @@ public Optional> tryReserve(FateId fateId) { return Optional.empty(); } - @Override - public boolean isReserved(FateId fateId) { - try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { - scanner.setRange(getRow(fateId)); - scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); - return scanner.stream() - .map(entry -> FateReservation.isFateReservation(entry.getValue().toString())).findFirst() - .orElse(false); - } catch (TableNotFoundException e) { - throw new IllegalStateException(tableName + " not found!", e); - } - } - - @Override - public Map getActiveReservations() { - Map activeReservations = new HashMap<>(); - - try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { - scanner.setRange(new Range()); - scanner.fetchColumn(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), - TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); - scanner.stream() - .filter(entry -> FateReservation.isFateReservation(entry.getValue().toString())) - .forEach(entry -> { - String reservationColVal = entry.getValue().toString(); - FateId fateId = FateId.from(fateInstanceType, entry.getKey().getRow().toString()); - FateReservation reservation = FateReservation.from(reservationColVal); - activeReservations.put(fateId, reservation); - }); - } catch (TableNotFoundException e) { - throw new IllegalStateException(tableName + " not found!", e); - } - - return activeReservations; - } - @Override public void deleteDeadReservations() { - for (Entry entry : getActiveReservations().entrySet()) { - FateId fateId = entry.getKey(); - FateReservation reservation = entry.getValue(); + for (Entry activeRes : getActiveReservations().entrySet()) { + FateId fateId = activeRes.getKey(); + FateReservation reservation = activeRes.getValue(); if (!isLockHeld.test(reservation.getLockID())) { - newMutator(fateId).putUnreserveTx(reservation).tryMutate(); - // No need to check the status... If it is ACCEPTED, we have successfully unreserved + var status = newMutator(fateId).putUnreserveTx(reservation).tryMutate(); + if (status == FateMutator.Status.ACCEPTED) { + // Technically, this should also be logged for the case where the mutation status + // is UNKNOWN, but the mutation was actually written (fate id was unreserved) + // but there is no way to tell if it was unreserved from this mutation or another + // thread simply unreserving the transaction + log.trace("Deleted the dead reservation {} for fate id {}", reservation, fateId); + } + // No need to verify the status... If it is ACCEPTED, we have successfully unreserved // the dead transaction. If it is REJECTED, the reservation has changed (i.e., // has been unreserved so no need to do anything, or has been unreserved and reserved // again in which case we don't want to change it). If it is UNKNOWN, the mutation @@ -252,14 +275,53 @@ protected Stream getTransactions(Set statuses) { Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY); scanner.setRange(new Range()); FateStatusFilter.configureScanner(scanner, statuses); + scanner.addScanIterator(new IteratorSetting(101, WholeRowIterator.class)); TxColumnFamily.STATUS_COLUMN.fetch(scanner); + TxColumnFamily.RESERVATION_COLUMN.fetch(scanner); return scanner.stream().onClose(scanner::close).map(e -> { String txUUIDStr = e.getKey().getRow().toString(); FateId fateId = FateId.from(fateInstanceType, txUUIDStr); + SortedMap rowMap; + TStatus status = TStatus.UNKNOWN; + FateReservation reservation = null; + try { + rowMap = WholeRowIterator.decodeRow(e.getKey(), e.getValue()); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + // expect status and optionally reservation + Preconditions.checkState(rowMap.size() == 1 || rowMap.size() == 2, + "Invalid row seen: %s. Expected to see one entry for the status and optionally an " + + "entry for the fate reservation", + rowMap); + for (Map.Entry entry : rowMap.entrySet()) { + Text colf = entry.getKey().getColumnFamily(); + Text colq = entry.getKey().getColumnQualifier(); + Value val = entry.getValue(); + switch (colq.toString()) { + case TxColumnFamily.STATUS: + status = TStatus.valueOf(val.toString()); + break; + case TxColumnFamily.RESERVATION: + if (FateReservation.isFateReservation(val.get())) { + reservation = FateReservation.deserialize(val.get()); + } + break; + default: + throw new IllegalStateException("Unexpected column seen: " + colf + ":" + colq); + } + } + final TStatus finalStatus = status; + final Optional finalReservation = Optional.ofNullable(reservation); return new FateIdStatusBase(fateId) { @Override public TStatus getStatus() { - return TStatus.valueOf(e.getValue().toString()); + return finalStatus; + } + + @Override + public Optional getFateReservation() { + return finalReservation; } }; }); @@ -301,35 +363,6 @@ protected Optional getKey(FateId fateId) { }); } - @Override - protected Pair> getStatusAndKey(FateId fateId) { - return scanTx(scanner -> { - scanner.setRange(getRow(fateId)); - TxColumnFamily.STATUS_COLUMN.fetch(scanner); - TxColumnFamily.TX_KEY_COLUMN.fetch(scanner); - - TStatus status = null; - FateKey key = null; - - for (Entry entry : scanner) { - final String qual = entry.getKey().getColumnQualifierData().toString(); - switch (qual) { - case TxColumnFamily.STATUS: - status = TStatus.valueOf(entry.getValue().toString()); - break; - case TxColumnFamily.TX_KEY: - key = FateKey.deserialize(entry.getValue().get()); - break; - default: - throw new IllegalStateException("Unexpected column qualifier: " + qual); - } - } - - return new Pair<>(Optional.ofNullable(status).orElse(TStatus.UNKNOWN), - Optional.ofNullable(key)); - }); - } - @Override protected FateTxStore newUnreservedFateTxStore(FateId fateId) { return new FateTxStoreImpl(fateId); diff --git a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java index 2acc6e41fac..5722c617236 100644 --- a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java +++ b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java @@ -177,11 +177,6 @@ public Optional> createAndReserve(FateKey fateKey) { return txStore; } - @Override - public boolean isReserved(FateId fateId) { - return store.isReserved(fateId); - } - @Override public Map getActiveReservations() { return store.getActiveReservations(); diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 632fba88685..ccefb8ea660 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -60,7 +60,8 @@ public Optional> createAndReserve(FateKey key) { @Override public FateTxStore reserve(FateId fateId) { if (reserved.contains(fateId)) { - throw new IllegalStateException(); // zoo store would wait, but do not expect test to reserve + // other fate stores would wait, but do not expect test to reserve + throw new IllegalStateException(); } // twice... if test change, then change this reserved.add(fateId); @@ -78,10 +79,6 @@ public Optional> tryReserve(FateId fateId) { } } - public boolean isReserved(FateId fateId) { - return reserved.contains(fateId); - } - @Override public Map getActiveReservations() { // This method only makes sense for the FateStores that don't store their reservations in memory @@ -114,26 +111,28 @@ public List> getStack() { @Override public TStatus getStatus() { - return getStatusAndKey().getFirst(); - } + if (!reserved.contains(fateId)) { + throw new IllegalStateException(); + } - @Override - public Optional getKey() { - return getStatusAndKey().getSecond(); + Pair> status = statuses.get(fateId); + if (status == null) { + return TStatus.UNKNOWN; + } + return status.getFirst(); } @Override - public Pair> getStatusAndKey() { + public Optional getKey() { if (!reserved.contains(fateId)) { throw new IllegalStateException(); } Pair> status = statuses.get(fateId); if (status == null) { - return new Pair<>(TStatus.UNKNOWN, Optional.empty()); + return Optional.empty(); } - - return status; + return status.getSecond(); } @Override @@ -225,6 +224,12 @@ public FateId getFateId() { public TStatus getStatus() { return e.getValue().getFirst(); } + + @Override + public Optional getFateReservation() { + throw new UnsupportedOperationException( + "Only the 'reserved' set should be used for reservations in the test store"); + } }); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java index 04bff32cd24..418223f7577 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java @@ -37,6 +37,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -685,11 +686,17 @@ protected void testFatePrintAndSummaryCommandsWithInProgressTxns(FateStore getFateReservation() { + return Optional.empty(); + } }; } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java index 2ffdb59031c..a2d66bcb929 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java @@ -55,7 +55,6 @@ import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.StackOverflowException; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; -import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateIT.TestRepo; @@ -69,6 +68,7 @@ public abstract class FateStoreIT extends SharedMiniClusterBase implements FateTestRunner { + // TODO 4131 this test will have to be changed since there is no longer a create(FateKey) method private static final Method fsCreateByKeyMethod; static { @@ -480,7 +480,6 @@ protected void testAbsent(FateStore store, ServerContext sctx) { assertEquals(Optional.empty(), txStore.getKey()); assertEquals(fateId, txStore.getID()); assertEquals(List.of(), txStore.getStack()); - assertEquals(new Pair<>(TStatus.UNKNOWN, Optional.empty()), txStore.getStatusAndKey()); } @Test diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java index 5584567cd59..382b8004c09 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java @@ -197,9 +197,9 @@ public void testReservations() throws Exception { assertEquals(REJECTED, status); // Initialize the column and ensure we can't do it twice - status = new FateMutatorImpl<>(context, table, fateId).putInitReserveColVal().tryMutate(); + status = new FateMutatorImpl<>(context, table, fateId).putInitReservationVal().tryMutate(); assertEquals(ACCEPTED, status); - status = new FateMutatorImpl<>(context, table, fateId).putInitReserveColVal().tryMutate(); + status = new FateMutatorImpl<>(context, table, fateId).putInitReservationVal().tryMutate(); assertEquals(REJECTED, status); // Ensure that reserving is the only thing we can do From d352678308ecebc32ec71f9f1035709780cf7ce8 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Wed, 24 Jul 2024 15:15:16 -0400 Subject: [PATCH 11/17] Fixed UserFateStore.createAndReserve() - New createAndReserve was not working as expected for UserFateStore, fixed the bug --- .../core/fate/user/UserFateStore.java | 43 +++++++++++++------ .../accumulo/test/fate/FateOpsCommandsIT.java | 14 +++--- 2 files changed, 38 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index 6b2325bd32f..2a39b2a7e7b 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.List; -import java.util.Map; import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; @@ -132,23 +131,39 @@ public Optional> createAndReserve(FateKey fateKey) { final var reservation = FateReservation.from(lockID, UUID.randomUUID()); final var fateId = fateIdGenerator.fromTypeAndKey(type(), fateKey); Optional> txStore = Optional.empty(); + int maxAttempts = 5; + FateMutator.Status status = null; - var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) - .putInitReservationVal().putReservedTx(reservation) - .putCreateTime(System.currentTimeMillis()).tryMutate(); + // We first need to write the initial/unreserved value for the reservation column + // Only need to retry if it is UNKNOWN + for (int attempt = 0; attempt < maxAttempts; attempt++) { + status = newMutator(fateId).putInitReservationVal().tryMutate(); + if (status != FateMutator.Status.UNKNOWN) { + break; + } + UtilWaitThread.sleep(100); + } + for (int attempt = 0; attempt < maxAttempts; attempt++) { + status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) + .putReservedTx(reservation).putCreateTime(System.currentTimeMillis()).tryMutate(); + if (status != FateMutator.Status.UNKNOWN) { + break; + } + UtilWaitThread.sleep(100); + } switch (status) { case ACCEPTED: txStore = Optional.of(new FateTxStoreImpl(fateId, reservation)); break; - case UNKNOWN: case REJECTED: - // If the status is UNKNOWN, the mutation may or may not have been written. We need to - // check if it was written only returning the FateTxStore if it was. // If the status is REJECTED, we need to check what about the mutation was REJECTED: - // 1) If there is a collision with existing fate id, throw error - // 2) If the fate id is already reserved, return an empty optional - // 3) If the fate id is still NEW/unseeded and unreserved, we can try to reserve it + // 1) Possible something like the following occurred: + // the first attempt was UNKNOWN but written, the next attempt would be rejected + // We return the FateTxStore in this case. + // 2) If there is a collision with existing fate id, throw error + // 3) If the fate id is already reserved, return an empty optional + // 4) If the fate id is still NEW/unseeded and unreserved, we can try to reserve it try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY)) { scanner.setRange(getRow(fateId)); scanner.fetchColumn(TxColumnFamily.STATUS_COLUMN.getColumnFamily(), @@ -161,7 +176,7 @@ public Optional> createAndReserve(FateKey fateKey) { Optional fateKeySeen = Optional.empty(); Optional reservationSeen = Optional.empty(); - for (Map.Entry entry : scanner.stream().collect(Collectors.toList())) { + for (Entry entry : scanner.stream().collect(Collectors.toList())) { Text colf = entry.getKey().getColumnFamily(); Text colq = entry.getKey().getColumnQualifier(); Value val = entry.getValue(); @@ -183,7 +198,7 @@ public Optional> createAndReserve(FateKey fateKey) { } } - // This will be the case if the mutation status is UNKNOWN but the mutation was written + // This will be the case if the mutation status is REJECTED but the mutation was written if (statusSeen == TStatus.NEW && reservationSeen.isPresent() && reservationSeen.orElseThrow().equals(reservation)) { verifyFateKey(fateId, fateKeySeen, fateKey); @@ -204,7 +219,7 @@ public Optional> createAndReserve(FateKey fateKey) { } break; default: - throw new IllegalStateException("Unknown status " + status); + throw new IllegalStateException("Unknown or unexpected status " + status); } return txStore; @@ -294,7 +309,7 @@ protected Stream getTransactions(Set statuses) { "Invalid row seen: %s. Expected to see one entry for the status and optionally an " + "entry for the fate reservation", rowMap); - for (Map.Entry entry : rowMap.entrySet()) { + for (Entry entry : rowMap.entrySet()) { Text colf = entry.getKey().getColumnFamily(); Text colq = entry.getKey().getColumnQualifier(); Value val = entry.getValue(); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java index 418223f7577..4fbfe407ada 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java @@ -41,6 +41,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.accumulo.core.Constants; @@ -61,6 +62,7 @@ import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.minicluster.ServerType; import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo; @@ -623,14 +625,16 @@ protected void testFatePrintAndSummaryCommandsWithInProgressTxns(FateStore Date: Thu, 1 Aug 2024 13:45:57 -0400 Subject: [PATCH 12/17] Bug fixes and code quality improvements: - Replaced static vars used to determine whether to run the dead reservation cleaner in Fate with a new parameter to Fate - Changed AbstractFateStore constructor to only accept non-null values for the LockID, and instead have the caller create a dummy LockID if needed - Fixed a case for MetaFateStore.tryReserve() where an exception would be thrown when instead an empty optional should be returned. This also keeps the functionality of User/MetaFateStore.tryReserve() the same, which they weren't entirely before this change. Also added check for this equivalent functionality in MultipleStoresIT.testReserveUnreserve(). - Fixed issue with MetaFateStore.deleteDeadReservations() where a NoNodeException was thrown when it shouldn't be - Added fate id to error message for MetaFateStore.setStatus() - Trivial simplification to MetaFateStore.getTransactions(): return whole node instead of a pair of the needed fields - Added check to MetaFateStore.NodeValue.deserializeFateKey() and deserializeFateReservation() to ensure valid non-negative length - Reworked FateStatusFilter (now RowFateStatusFilter) to work with the whole row instead of individual Key/Value pairs - Fixed UserFateStore.createAndReserve() to only write one mutation instead of two. This required adding a new method FateMutator.putReservedTxOnCreation() - Added check to UserFateStore.createAndReserve() needed to avoid a potential race condition - Fixed FateStoreIT to work with new implementation of createAndReserve(). A method was deleted that was no longer used which FateStoreIT relied on. - Upon fixing FateStoreIT, noticed that the original implementation of createAndReserve expected an error to be thrown in a certain situation, but in the new implementation, an empty optional was returned instead. Fixed the new implementation to match the previous implementation for this case. --- .../accumulo/core/fate/AbstractFateStore.java | 11 ++-- .../org/apache/accumulo/core/fate/Fate.java | 37 +++----------- .../accumulo/core/fate/MetaFateStore.java | 47 ++++++++++------- .../accumulo/core/fate/user/FateMutator.java | 10 ++++ .../core/fate/user/FateMutatorImpl.java | 9 ++++ ...usFilter.java => RowFateStatusFilter.java} | 30 ++++++----- .../core/fate/user/UserFateStore.java | 48 +++++++++--------- .../apache/accumulo/server/util/Admin.java | 9 ++-- .../org/apache/accumulo/manager/Manager.java | 2 +- .../manager/metrics/fate/FateMetrics.java | 5 +- .../compaction/ExternalCompaction_1_IT.java | 14 +++--- .../org/apache/accumulo/test/fate/FateIT.java | 2 +- .../test/fate/FateInterleavingIT.java | 2 +- .../accumulo/test/fate/FateOpsCommandsIT.java | 9 ++-- .../accumulo/test/fate/FateStoreIT.java | 50 +++++-------------- .../apache/accumulo/test/fate/FlakyFate.java | 2 +- .../accumulo/test/fate/MultipleStoresIT.java | 15 +++--- .../accumulo/test/fate/meta/MetaFateIT.java | 5 +- .../fate/meta/MetaFateInterleavingIT.java | 4 +- .../test/fate/meta/MetaFateOpsCommandsIT.java | 4 +- .../test/fate/meta/MetaFateStoreFateIT.java | 5 +- .../test/fate/user/FateMutatorImplIT.java | 18 ++++++- .../accumulo/test/fate/user/UserFateIT.java | 6 +-- .../fate/user/UserFateInterleavingIT.java | 6 +-- .../test/fate/user/UserFateOpsCommandsIT.java | 5 +- .../test/fate/user/UserFateStoreFateIT.java | 6 +-- .../test/fate/user/UserFateStoreIT.java | 2 +- .../test/functional/FateConcurrencyIT.java | 13 ++--- .../test/functional/FunctionalTestUtils.java | 7 +-- 29 files changed, 202 insertions(+), 181 deletions(-) rename core/src/main/java/org/apache/accumulo/core/fate/user/{FateStatusFilter.java => RowFateStatusFilter.java} (74%) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index ab90dbd2269..257b724b2d0 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -57,8 +57,8 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; // TODO 4131 should probably add support to AbstractFateStore, MetaFateStore, -// and UserFateStore to accept null lockID and zooCache (maybe make these fields -// Optional<>). This could replace the current createDummyLockID(). This support +// and UserFateStore to accept null lockID (maybe make this field Optional<>). +// This could replace the current createDummyLockID(). This support // is needed since MFS and UFS aren't always created in the context of a Manager. public abstract class AbstractFateStore implements FateStore { @@ -91,7 +91,7 @@ public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { private final AtomicInteger concurrentStatusChangeCallers = new AtomicInteger(0); public AbstractFateStore() { - this(null, null, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); + this(createDummyLockID(), null, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } public AbstractFateStore(ZooUtil.LockID lockID, Predicate isLockHeld) { @@ -103,8 +103,9 @@ public AbstractFateStore(ZooUtil.LockID lockID, Predicate isLock this.maxDeferred = maxDeferred; this.fateIdGenerator = Objects.requireNonNull(fateIdGenerator); this.deferred = Collections.synchronizedMap(new HashMap<>()); - this.lockID = Objects.requireNonNullElseGet(lockID, AbstractFateStore::createDummyLockID); - // If the store is used for a Fate object, this should be non-null, otherwise null is fine + this.lockID = Objects.requireNonNull(lockID); + // If the store is used for a Fate which runs a dead reservation cleaner, + // this should be non-null, otherwise null is fine this.isLockHeld = isLockHeld; } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index 5ed9b090165..21ca036cd23 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -79,8 +79,6 @@ public class Fate { private final ExecutorService deadResCleanerExecutor; private static final EnumSet FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); - private static boolean userDeadReservationCleanerRunning = false; - private static boolean metaDeadReservationCleanerRunning = false; private final AtomicBoolean keepRunning = new AtomicBoolean(true); private final TransferQueue workQueue; @@ -329,10 +327,7 @@ protected Repo executeCall(FateId fateId, Repo op) throws Exception { } /** - * A thread that finds reservations held by dead processes and unreserves them. Only one thread - * runs per store type across all Fate instances (one to clean up dead reservations for - * {@link org.apache.accumulo.core.fate.user.UserFateStore UserFateStore} and one to clean up dead - * reservations for {@link MetaFateStore}). + * A thread that finds reservations held by dead processes and unreserves them */ private class DeadReservationCleaner implements Runnable { @Override @@ -348,8 +343,8 @@ public void run() { * * @param toLogStrFunc A function that converts Repo to Strings that are suitable for logging */ - public Fate(T environment, FateStore store, Function,String> toLogStrFunc, - AccumuloConfiguration conf) { + public Fate(T environment, FateStore store, boolean runDeadResCleaner, + Function,String> toLogStrFunc, AccumuloConfiguration conf) { this.store = FateLogger.wrap(store, toLogStrFunc); this.environment = environment; final ThreadPoolExecutor pool = ThreadPools.getServerThreadPools().createExecutorService(conf, @@ -382,26 +377,17 @@ public Fate(T environment, FateStore store, Function,String> toLogStr }, 3, SECONDS)); this.transactionExecutor = pool; - // Create a dead reservation cleaner for this store that will periodically (every 30 seconds) - // clean up reservations held by dead processes, if they exist. Only created if a dead - // reservation cleaner is not already running for the given store type. - // TODO 4131 periodic cleanup runs every 30 seconds - // Should this be longer? Shorter? A configurable Property? A function of something? ScheduledExecutorService deadResCleanerExecutor = null; - boolean isUserStore = store.type() == FateInstanceType.USER; - boolean isMetaStore = store.type() == FateInstanceType.META; - if ((isUserStore && !userDeadReservationCleanerRunning) - || (isMetaStore && !metaDeadReservationCleanerRunning)) { + if (runDeadResCleaner) { + // Create a dead reservation cleaner for this store that will periodically (every 30 seconds) + // clean up reservations held by dead processes, if they exist. deadResCleanerExecutor = ThreadPools.getServerThreadPools().createScheduledExecutorService(1, store.type() + "-dead-reservation-cleaner-pool"); + // TODO 4131 periodic cleanup runs every 30 seconds + // Should this be longer? Shorter? A configurable Property? A function of something? ScheduledFuture deadReservationCleaner = deadResCleanerExecutor .scheduleWithFixedDelay(new DeadReservationCleaner(), 3, 30, SECONDS); ThreadPools.watchCriticalScheduledTask(deadReservationCleaner); - if (isUserStore) { - userDeadReservationCleanerRunning = true; - } else { - metaDeadReservationCleanerRunning = true; - } } this.deadResCleanerExecutor = deadResCleanerExecutor; @@ -624,12 +610,5 @@ public void shutdown(long timeout, TimeUnit timeUnit) { if (deadResCleanerExecutor != null) { deadResCleanerExecutor.shutdownNow(); } - - // Update that USER/META dead reservation cleaner is no longer running - if (store.type() == FateInstanceType.USER && userDeadReservationCleanerRunning) { - userDeadReservationCleanerRunning = false; - } else if (store.type() == FateInstanceType.META && metaDeadReservationCleanerRunning) { - metaDeadReservationCleanerRunning = false; - } } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index 5d998c25a11..69b9bab0818 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -45,7 +45,6 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; -import org.apache.accumulo.core.util.Pair; import org.apache.hadoop.io.DataInputBuffer; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -55,6 +54,7 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; //TODO use zoocache? - ACCUMULO-1297 @@ -125,21 +125,25 @@ public Optional> createAndReserve(FateKey fateKey) { // 2) The existing NodeValue for fateId has: // TStatus = TStatus.NEW, no FateReservation present, FateKey = fateKey // The fateId is NEW/unseeded and not reserved, so we can allow it to be reserved + // Note: returning null here will not change the value to null but will return null NodeValue currNodeVal = new NodeValue(currSerNodeVal); - if (currNodeVal.status == TStatus.NEW && currNodeVal.isReservedBy(reservation)) { + if (currNodeVal.status == TStatus.NEW) { verifyFateKey(fateId, currNodeVal.fateKey, fateKey); - return currSerNodeVal; - } else if (currNodeVal.status == TStatus.NEW && !currNodeVal.isReserved()) { - verifyFateKey(fateId, currNodeVal.fateKey, fateKey); - // NEW/unseeded transaction and not reserved, so we can allow it to be reserved - return new NodeValue(TStatus.NEW, reservation, fateKey).serialize(); + if (currNodeVal.isReservedBy(reservation)) { + return currSerNodeVal; + } else if (!currNodeVal.isReserved()) { + // NEW/unseeded transaction and not reserved, so we can allow it to be reserved + return new NodeValue(TStatus.NEW, reservation, fateKey).serialize(); + } else { + // NEW/unseeded transaction reserved under a different reservation + return null; + } } else { log.trace( - "fate id {} tstatus {} fate key {} is reserved {} is either currently reserved " - + "or has already been seeded with work (non-NEW status), or both", + "fate id {} tstatus {} fate key {} is reserved {} " + + "has already been seeded with work (non-NEW status)", fateId, currNodeVal.status, currNodeVal.fateKey.orElse(null), currNodeVal.isReserved()); - // This will not change the value to null but will return null return null; } }); @@ -178,6 +182,9 @@ public Optional> tryReserve(FateId fateId) { } else { return Optional.empty(); } + } catch (KeeperException.NoNodeException e) { + log.trace("Tried to reserve a transaction {} that does not exist", fateId); + return Optional.empty(); } catch (InterruptedException | KeeperException | AcceptableThriftTableOperationException e) { throw new IllegalStateException(e); } @@ -207,6 +214,8 @@ public void deleteDeadReservations() { return null; } }); + } catch (KeeperException.NoNodeException e) { + // the node has since been deleted. Can safely ignore } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { throw new RuntimeException(e); } @@ -331,8 +340,10 @@ public void setStatus(TStatus status) { NodeValue newNodeValue = new NodeValue(status, currFateReservation, currFateKey); return newNodeValue.serialize(); } else { - throw new IllegalStateException("Either the FateId is not reserved in ZK, or it is" - + " but the reservation in ZK differs from that in the store."); + throw new IllegalStateException("Either the FateId " + fateId + + " is not reserved in ZK, or it is but the reservation in ZK: " + + currNodeVal.reservation.orElse(null) + " differs from that in the store: " + + this.reservation); } }); } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { @@ -501,20 +512,16 @@ protected Stream getTransactions(Set statuses) { // Memoizing for two reasons. First the status or reservation may never be requested, so // in that case avoid the lookup. Second, if it's requested multiple times the result will // always be consistent. - Supplier>> statusAndResSupplier = - Suppliers.memoize(() -> { - NodeValue zkNode = getNode(fateId); - return new Pair<>(zkNode.status, zkNode.reservation); - }); + Supplier nodeSupplier = Suppliers.memoize(() -> getNode(fateId)); return new FateIdStatusBase(fateId) { @Override public TStatus getStatus() { - return statusAndResSupplier.get().getFirst(); + return nodeSupplier.get().status; } @Override public Optional getFateReservation() { - return statusAndResSupplier.get().getSecond(); + return nodeSupplier.get().reservation; } }; }); @@ -563,6 +570,7 @@ private NodeValue(TStatus status, FateReservation reservation, FateKey fateKey) private Optional deserializeFateKey(DataInputBuffer buffer) throws IOException { int length = buffer.readInt(); + Preconditions.checkArgument(length >= 0); if (length > 0) { return Optional.of(FateKey.deserialize(buffer.readNBytes(length))); } @@ -572,6 +580,7 @@ private Optional deserializeFateKey(DataInputBuffer buffer) throws IOEx private Optional deserializeFateReservation(DataInputBuffer buffer) throws IOException { int length = buffer.readInt(); + Preconditions.checkArgument(length >= 0); if (length > 0) { return Optional.of(FateReservation.deserialize(buffer.readNBytes(length))); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java index 067579f0069..8c39e897000 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java @@ -42,6 +42,16 @@ public interface FateMutator { */ FateMutator putReservedTx(FateStore.FateReservation reservation); + /** + * Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will + * put the reservation if the column doesn't exist yet. This should only be used for + * {@link UserFateStore#createAndReserve(FateKey)} + * + * @param reservation the reservation to attempt to put + * @return the FateMutator with this added mutation + */ + FateMutator putReservedTxOnCreation(FateStore.FateReservation reservation); + /** * Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will * remove the given reservation if it matches what is present in the column. diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java index e9a10c1229f..fcb0e4f1f16 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/FateMutatorImpl.java @@ -92,6 +92,15 @@ public FateMutator putReservedTx(FateStore.FateReservation reservation) { return this; } + @Override + public FateMutator putReservedTxOnCreation(FateStore.FateReservation reservation) { + Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), + TxColumnFamily.RESERVATION_COLUMN.getColumnQualifier()); + mutation.addCondition(condition); + TxColumnFamily.RESERVATION_COLUMN.put(mutation, new Value(reservation.getSerialized())); + return this; + } + @Override public FateMutator putUnreserveTx(FateStore.FateReservation reservation) { Condition condition = new Condition(TxColumnFamily.RESERVATION_COLUMN.getColumnFamily(), diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java b/core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java similarity index 74% rename from core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java rename to core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java index df179e383b8..fd4459837b0 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/FateStatusFilter.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.EnumSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -30,13 +31,14 @@ import org.apache.accumulo.core.client.ScannerBase; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; -import org.apache.accumulo.core.iterators.Filter; +import org.apache.accumulo.core.fate.user.schema.FateSchema; import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.user.WholeRowIterator; +import org.apache.hadoop.io.Text; -public class FateStatusFilter extends Filter { +public class RowFateStatusFilter extends WholeRowIterator { private EnumSet valuesToAccept; @@ -54,16 +56,16 @@ public void init(SortedKeyValueIterator source, Map op } @Override - public boolean accept(Key k, Value v) { - // We may see TStatus values or FateReservation values with how this filter is used, - // only accept TStatus values, return false on FateReservation values, error otherwise - try { - var tstatus = ReadOnlyFateStore.TStatus.valueOf(v.toString()); - return valuesToAccept.contains(tstatus); - } catch (IllegalArgumentException e) { - FateStore.FateReservation.isFateReservation(v.get()); - return false; + protected boolean filter(Text currentRow, List keys, List values) { + for (int i = 0; i < keys.size(); i++) { + Key key = keys.get(i); + if (key.getColumnQualifier() + .equals(FateSchema.TxColumnFamily.STATUS_COLUMN.getColumnQualifier()) + && valuesToAccept.contains(ReadOnlyFateStore.TStatus.valueOf(values.get(i).toString()))) { + return true; + } } + return false; } public static void configureScanner(ScannerBase scanner, @@ -71,9 +73,11 @@ public static void configureScanner(ScannerBase scanner, // only filter when getting a subset of statuses if (!statuses.equals(ALL_STATUSES)) { String statusesStr = statuses.stream().map(Enum::name).collect(Collectors.joining(",")); - var iterSettings = new IteratorSetting(100, "statuses", FateStatusFilter.class); + var iterSettings = new IteratorSetting(100, "statuses", RowFateStatusFilter.class); iterSettings.addOption("statuses", statusesStr); scanner.addScanIterator(iterSettings); + } else { + scanner.addScanIterator(new IteratorSetting(100, WholeRowIterator.class)); } } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java index 2a39b2a7e7b..44813c2c9a0 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.Serializable; +import java.time.Duration; import java.util.List; import java.util.Map.Entry; import java.util.Objects; @@ -32,7 +33,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; @@ -134,18 +134,11 @@ public Optional> createAndReserve(FateKey fateKey) { int maxAttempts = 5; FateMutator.Status status = null; - // We first need to write the initial/unreserved value for the reservation column // Only need to retry if it is UNKNOWN - for (int attempt = 0; attempt < maxAttempts; attempt++) { - status = newMutator(fateId).putInitReservationVal().tryMutate(); - if (status != FateMutator.Status.UNKNOWN) { - break; - } - UtilWaitThread.sleep(100); - } for (int attempt = 0; attempt < maxAttempts; attempt++) { status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) - .putReservedTx(reservation).putCreateTime(System.currentTimeMillis()).tryMutate(); + .putReservedTxOnCreation(reservation).putCreateTime(System.currentTimeMillis()) + .tryMutate(); if (status != FateMutator.Status.UNKNOWN) { break; } @@ -176,7 +169,7 @@ public Optional> createAndReserve(FateKey fateKey) { Optional fateKeySeen = Optional.empty(); Optional reservationSeen = Optional.empty(); - for (Entry entry : scanner.stream().collect(Collectors.toList())) { + for (Entry entry : scanner) { Text colf = entry.getKey().getColumnFamily(); Text colq = entry.getKey().getColumnQualifier(); Value val = entry.getValue(); @@ -198,20 +191,28 @@ public Optional> createAndReserve(FateKey fateKey) { } } - // This will be the case if the mutation status is REJECTED but the mutation was written - if (statusSeen == TStatus.NEW && reservationSeen.isPresent() - && reservationSeen.orElseThrow().equals(reservation)) { - verifyFateKey(fateId, fateKeySeen, fateKey); - txStore = Optional.of(new FateTxStoreImpl(fateId, reservation)); - } else if (statusSeen == TStatus.NEW && reservationSeen.isEmpty()) { + if (statusSeen == TStatus.NEW) { verifyFateKey(fateId, fateKeySeen, fateKey); - // NEW/unseeded transaction and not reserved, so we can allow it to be reserved - // we tryReserve() since another thread may have reserved it since the scan - txStore = tryReserve(fateId); + // This will be the case if the mutation status is REJECTED but the mutation was written + if (reservationSeen.isPresent() && reservationSeen.orElseThrow().equals(reservation)) { + txStore = Optional.of(new FateTxStoreImpl(fateId, reservation)); + } else if (reservationSeen.isEmpty()) { + // NEW/unseeded transaction and not reserved, so we can allow it to be reserved + // we tryReserve() since another thread may have reserved it since the scan + txStore = tryReserve(fateId); + // the status was known before reserving to be NEW, + // however it could change so check after reserving to avoid race conditions. + var statusAfterReserve = + txStore.map(ReadOnlyFateTxStore::getStatus).orElse(TStatus.UNKNOWN); + if (statusAfterReserve != TStatus.NEW) { + txStore.ifPresent(txs -> txs.unreserve(Duration.ZERO)); + txStore = Optional.empty(); + } + } } else { log.trace( - "fate id {} tstatus {} fate key {} is reserved {} is either currently reserved " - + "or has already been seeded with work (non-NEW status), or both", + "fate id {} tstatus {} fate key {} is reserved {} " + + "has already been seeded with work (non-NEW status)", fateId, statusSeen, fateKeySeen.orElse(null), reservationSeen.isPresent()); } } catch (TableNotFoundException e) { @@ -289,8 +290,7 @@ protected Stream getTransactions(Set statuses) { try { Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY); scanner.setRange(new Range()); - FateStatusFilter.configureScanner(scanner, statuses); - scanner.addScanIterator(new IteratorSetting(101, WholeRowIterator.class)); + RowFateStatusFilter.configureScanner(scanner, statuses); TxColumnFamily.STATUS_COLUMN.fetch(scanner); TxColumnFamily.RESERVATION_COLUMN.fetch(scanner); return scanner.stream().onClose(scanner::close).map(e -> { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index 00ad73a9596..9839d3b059e 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import java.io.BufferedWriter; import java.io.File; @@ -805,8 +806,8 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore mfs = new MetaFateStore<>(fateZkPath, zk, null, null); - UserFateStore ufs = new UserFateStore<>(context, null, null); + MetaFateStore mfs = new MetaFateStore<>(fateZkPath, zk, createDummyLockID(), null); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); Map> readOnlyFateStores = @@ -1000,9 +1001,9 @@ private static long printDanglingFateOperations(ServerContext context, } }; - UserFateStore ufs = new UserFateStore<>(context, null, null); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); MetaFateStore mfs = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter(), null, null); + context.getZooReaderWriter(), createDummyLockID(), null); LoadingCache fateStatusCache = Caffeine.newBuilder() .maximumSize(100_000).expireAfterWrite(10, TimeUnit.SECONDS).build(fateId -> { if (fateId.getType() == FateInstanceType.META) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index 6cae916986e..5b9984a45e5 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -1355,7 +1355,7 @@ boolean canSuspendTablets() { protected Fate initializeFateInstance(ServerContext context, FateStore store) { final Fate fateInstance = - new Fate<>(this, store, TraceRepo::toLogString, getConfiguration()); + new Fate<>(this, store, true, TraceRepo::toLogString, getConfiguration()); var fateCleaner = new FateCleaner<>(store, Duration.ofHours(8), this::getSteadyTime); ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java index 3a27a13a8ed..cebc77a39d5 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.manager.metrics.fate; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; + import java.util.List; import java.util.Map.Entry; import java.util.concurrent.ScheduledExecutorService; @@ -72,7 +74,8 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter(), null, null); + this.fateStore = new MetaFateStore<>(fateRootPath, context.getZooReaderWriter(), + createDummyLockID(), null); } catch (KeeperException ex) { throw new IllegalStateException( "FATE Metrics - Failed to create zoo store - metrics unavailable", ex); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index b1ca3340d82..314212693a3 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.compaction; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP1; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP2; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP3; @@ -233,7 +234,7 @@ public void testExternalCompaction() throws Exception { public void testCompactionCommitAndDeadDetectionRoot() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), null, null); + ctx.getZooReaderWriter(), createDummyLockID(), null); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var tableId = ctx.getTableId(AccumuloTable.ROOT.tableName()); @@ -252,7 +253,7 @@ public void testCompactionCommitAndDeadDetectionRoot() throws Exception { public void testCompactionCommitAndDeadDetectionMeta() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), null, null); + ctx.getZooReaderWriter(), createDummyLockID(), null); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { // Metadata table by default already has 2 tablets @@ -274,7 +275,7 @@ public void testCompactionCommitAndDeadDetectionUser() throws Exception { final String tableName = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { - UserFateStore userFateStore = new UserFateStore<>(ctx, null, null); + UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID(), null); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); c.tableOperations().create(tableName, new NewTableConfiguration().withSplits(splits)); @@ -297,9 +298,10 @@ public void testCompactionCommitAndDeadDetectionAll() throws Exception { final String userTable = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { - UserFateStore userFateStore = new UserFateStore<>(ctx, null, null); - FateStore metaFateStore = new MetaFateStore<>( - ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), null, null); + UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID(), null); + FateStore metaFateStore = + new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), + createDummyLockID(), null); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java index 00847ad5f78..bd7c4a2395b 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java @@ -492,7 +492,7 @@ protected Fate initializeFate(FateStore store) { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - return new Fate<>(new TestEnv(), store, r -> r + "", config); + return new Fate<>(new TestEnv(), store, false, r -> r + "", config); } protected abstract TStatus getTxStatus(ServerContext sctx, FateId fateId); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateInterleavingIT.java index 7c46f8a75ea..b4bf4146829 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateInterleavingIT.java @@ -189,7 +189,7 @@ protected Fate initializeFate(AccumuloClient client, FateStore(new FilTestEnv(client), store, r -> r + "", config); + return new Fate<>(new FilTestEnv(client), store, false, r -> r + "", config); } private static Entry toIdStep(Entry e) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java index 4fbfe407ada..8b52f88f97f 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.util.compaction.ExternalCompactionUtil.getCompactorAddrs; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; @@ -627,14 +628,14 @@ protected void testFatePrintAndSummaryCommandsWithInProgressTxns(FateStore initializeFate(FateStore store) { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - return new Fate<>(new TestEnv(), store, Object::toString, config); + return new Fate<>(new TestEnv(), store, false, Object::toString, config); } private boolean wordIsTStatus(String word) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java index a2d66bcb929..a2c2159dcfb 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java @@ -28,7 +28,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import java.lang.reflect.Method; import java.time.Duration; import java.util.HashMap; import java.util.HashSet; @@ -63,24 +62,10 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; -import com.google.common.base.Throwables; import com.google.common.collect.Sets; public abstract class FateStoreIT extends SharedMiniClusterBase implements FateTestRunner { - // TODO 4131 this test will have to be changed since there is no longer a create(FateKey) method - private static final Method fsCreateByKeyMethod; - - static { - try { - // Private method, need to capture for testing - fsCreateByKeyMethod = AbstractFateStore.class.getDeclaredMethod("create", FateKey.class); - fsCreateByKeyMethod.setAccessible(true); - } catch (NoSuchMethodException e) { - throw new RuntimeException(e); - } - } - @Override protected Duration defaultTimeout() { return Duration.ofMinutes(1); @@ -376,7 +361,7 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte // We have an existing transaction with the same key in progress // so should return an empty Optional - assertTrue(create(store, fateKey).isEmpty()); + assertTrue(store.createAndReserve(fateKey).isEmpty()); assertEquals(TStatus.IN_PROGRESS, txStore.getStatus()); } finally { txStore.setStatus(TStatus.SUCCESSFUL); @@ -415,9 +400,10 @@ protected void testCreateWithKeyCollision(FateStore store, ServerContex FateTxStore txStore = store.createAndReserve(fateKey1).orElseThrow(); try { - var e = assertThrows(IllegalStateException.class, () -> create(store, fateKey2)); + var e = assertThrows(IllegalStateException.class, () -> store.createAndReserve(fateKey2)); assertEquals( - "Collision detected for tid " + UUID.nameUUIDFromBytes("testing uuid".getBytes(UTF_8)), + "Collision detected for fate id " + + FateId.from(store.type(), UUID.nameUUIDFromBytes("testing uuid".getBytes(UTF_8))), e.getMessage()); assertEquals(fateKey1, txStore.getKey().orElseThrow()); } finally { @@ -438,19 +424,18 @@ protected void testCollisionWithRandomFateId(FateStore store, ServerCon new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); FateKey fateKey = FateKey.forSplit(ke); - FateId fateId = create(store, fateKey).orElseThrow(); + FateTxStore txStore = store.createAndReserve(fateKey).orElseThrow(); + FateId fateId = txStore.getID(); - // After create a fate transaction using a key we can simulate a collision with - // a random FateId by deleting the key out of Fate and calling create again to verify - // it detects the key is missing. Then we can continue and see if we can still reserve - // and use the existing transaction, which we should. + // After createAndReserve a fate transaction using a key we can simulate a collision with + // a random FateId by deleting the key out of Fate and calling createAndReserve again to + // verify it detects the key is missing. Then we can continue and see if we can still use + // the existing transaction. deleteKey(fateId, sctx); var e = assertThrows(IllegalStateException.class, () -> store.createAndReserve(fateKey)); - assertEquals("Tx Key is missing from tid " + fateId.getTxUUIDStr(), e.getMessage()); + assertEquals("fate key is missing from fate id " + fateId, e.getMessage()); - // We should still be able to reserve and continue when not using a key - // just like a normal transaction - FateTxStore txStore = store.reserve(fateId); + // We should still be able to use the existing transaction try { assertTrue(txStore.timeCreated() > 0); assertEquals(TStatus.NEW, txStore.getStatus()); @@ -546,17 +531,6 @@ protected void testListFateKeys(FateStore store, ServerContext sctx) th .forEach(fateIdStatus -> store.tryReserve(fateIdStatus.getFateId()).orElseThrow().delete()); } - // create(fateKey) method is private so expose for testing to check error states - @SuppressWarnings("unchecked") - protected Optional create(FateStore store, FateKey fateKey) throws Exception { - try { - return (Optional) fsCreateByKeyMethod.invoke(store, fateKey); - } catch (Exception e) { - Exception rootCause = (Exception) Throwables.getRootCause(e); - throw rootCause; - } - } - protected abstract void deleteKey(FateId fateId, ServerContext sctx); private static class TestOperation2 extends TestRepo { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FlakyFate.java b/test/src/main/java/org/apache/accumulo/test/fate/FlakyFate.java index 739b0cf0110..a5e75416b60 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FlakyFate.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FlakyFate.java @@ -35,7 +35,7 @@ public class FlakyFate extends Fate { public FlakyFate(T environment, FateStore store, Function,String> toLogStrFunc, AccumuloConfiguration conf) { - super(environment, store, toLogStrFunc, conf); + super(environment, store, false, toLogStrFunc, conf); } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index 3a70acb7cbc..221e5f597fb 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -113,6 +113,7 @@ private void testReserveUnreserve(FateInstanceType storeType) throws Exception { // reserving/unreserving a FateId should be reflected across instances of the stores final String tableName = getUniqueNames(1)[0]; final int numFateIds = 500; + final FateId fakeFateId = FateId.from(storeType, UUID.randomUUID()); final List> reservations = new ArrayList<>(); final boolean isUserStore = storeType.equals(FateInstanceType.USER); final Set allIds = new HashSet<>(); @@ -150,6 +151,9 @@ private void testReserveUnreserve(FateInstanceType storeType) throws Exception { } count++; } + // Try to reserve a non-existent fate id + assertTrue(store1.tryReserve(fakeFateId).isEmpty()); + assertTrue(store2.tryReserve(fakeFateId).isEmpty()); // Both stores should return the same reserved transactions activeReservations = store1.getActiveReservations(); assertEquals(allIds, activeReservations.keySet()); @@ -332,9 +336,9 @@ private void testMultipleFateInstances(FateInstanceType storeType) throws Except liveLocks.add(lock2); Fate fate1 = - new Fate<>(testEnv1, store1, Object::toString, DefaultConfiguration.getInstance()); + new Fate<>(testEnv1, store1, true, Object::toString, DefaultConfiguration.getInstance()); Fate fate2 = - new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); + new Fate<>(testEnv2, store2, false, Object::toString, DefaultConfiguration.getInstance()); for (int i = 0; i < numFateIds; i++) { FateId fateId; @@ -398,7 +402,7 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce liveLocks.add(lock1); Fate fate1 = - new Fate<>(testEnv1, store1, Object::toString, DefaultConfiguration.getInstance()); + new Fate<>(testEnv1, store1, true, Object::toString, DefaultConfiguration.getInstance()); // Ensure nothing is reserved yet assertTrue(store1.getActiveReservations().isEmpty()); @@ -442,10 +446,9 @@ private void testDeadReservationsCleanup(FateInstanceType storeType) throws Exce liveLocks.add(lock2); // Create the new Fate/start the Fate threads (the work finder and the workers). - // The DeadReservationCleaner for fate2 should not run/have no effect since we - // already have a DeadReservationCleaner for storeType running from fate1. + // Don't run another dead reservation cleaner since we already have one running from fate1. Fate fate2 = - new Fate<>(testEnv2, store2, Object::toString, DefaultConfiguration.getInstance()); + new Fate<>(testEnv2, store2, false, Object::toString, DefaultConfiguration.getInstance()); // Wait for the "dead" reservations to be deleted and picked up again (reserved using // fate2/store2/lock2 now). diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java index f8a3621d130..a23dde06448 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -75,8 +76,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, null, maxDeferred, - fateIdGenerator), sctx); + testMethod.execute(new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, createDummyLockID(), null, + maxDeferred, fateIdGenerator), sctx); } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java index 5df36693dab..bfd267630f5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; + import java.util.UUID; import org.apache.accumulo.core.Constants; @@ -39,6 +41,6 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred String path = ZK_ROOT + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); zk.mkdirs(ZK_ROOT); - testMethod.execute(new MetaFateStore<>(path, zk, null, null), sctx); + testMethod.execute(new MetaFateStore<>(path, zk, createDummyLockID(), null), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java index 13508054672..994c7af2ebe 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; + import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.MetaFateStore; @@ -32,6 +34,6 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, ServerContext sctx = getCluster().getServerContext(); String path = sctx.getZooKeeperRoot() + Constants.ZFATE; ZooReaderWriter zk = sctx.getZooReaderWriter(); - testMethod.execute(new MetaFateStore<>(path, zk, null, null), sctx); + testMethod.execute(new MetaFateStore<>(path, zk, createDummyLockID(), null), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java index 423009c7938..beb48a5304e 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.meta; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -77,8 +78,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, null, null, - maxDeferred, fateIdGenerator); + MetaFateStore store = new MetaFateStore<>(ZK_ROOT + Constants.ZFATE, zk, + createDummyLockID(), null, maxDeferred, fateIdGenerator); // Check that the store has no transactions before and after each test assertEquals(0, store.list().count()); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java index 382b8004c09..fe16e2b014d 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/FateMutatorImplIT.java @@ -180,8 +180,8 @@ public void testReservations() throws Exception { ClientContext context = (ClientContext) client; - FateId fateId = - FateId.from(FateInstanceType.fromNamespaceOrTableName(table), UUID.randomUUID()); + FateId fateId = FateId.from(FateInstanceType.USER, UUID.randomUUID()); + FateId fateId1 = FateId.from(FateInstanceType.USER, UUID.randomUUID()); ZooUtil.LockID lockID = new ZooUtil.LockID("/locks", "L1", 50); FateStore.FateReservation reservation = FateStore.FateReservation.from(lockID, UUID.randomUUID()); @@ -226,6 +226,20 @@ public void testReservations() throws Exception { status = new FateMutatorImpl<>(context, table, fateId).putUnreserveTx(reservation).tryMutate(); assertEquals(REJECTED, status); + + // Verify putReservedTxOnCreation works as expected + status = new FateMutatorImpl<>(context, table, fateId1).putReservedTxOnCreation(reservation) + .tryMutate(); + assertEquals(ACCEPTED, status); + status = new FateMutatorImpl<>(context, table, fateId1).putReservedTxOnCreation(reservation) + .tryMutate(); + assertEquals(REJECTED, status); + status = + new FateMutatorImpl<>(context, table, fateId1).putUnreserveTx(reservation).tryMutate(); + assertEquals(ACCEPTED, status); + status = new FateMutatorImpl<>(context, table, fateId1).putReservedTxOnCreation(reservation) + .tryMutate(); + assertEquals(REJECTED, status); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java index 36c1cc20bb2..7f0383e6f4c 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; import java.util.stream.StreamSupport; @@ -60,9 +61,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute( - new UserFateStore<>(client, table, null, null, maxDeferred, fateIdGenerator), - getCluster().getServerContext()); + testMethod.execute(new UserFateStore<>(client, table, createDummyLockID(), null, maxDeferred, + fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java index 7706604deed..83a87db975d 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateInterleavingIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; import org.apache.accumulo.core.client.Accumulo; @@ -34,9 +35,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute( - new UserFateStore<>(client, table, null, null, maxDeferred, fateIdGenerator), - getCluster().getServerContext()); + testMethod.execute(new UserFateStore<>(client, table, createDummyLockID(), null, maxDeferred, + fateIdGenerator), getCluster().getServerContext()); client.tableOperations().delete(table); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java index e203da4f7c7..3fe3192e6ba 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateOpsCommandsIT.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; + import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.test.fate.FateOpsCommandsIT; @@ -26,7 +28,8 @@ public class UserFateOpsCommandsIT extends FateOpsCommandsIT { @Override public void executeTest(FateTestExecutor testMethod, int maxDeferred, AbstractFateStore.FateIdGenerator fateIdGenerator) throws Exception { - testMethod.execute(new UserFateStore<>(getCluster().getServerContext(), null, null), + testMethod.execute( + new UserFateStore<>(getCluster().getServerContext(), createDummyLockID(), null), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java index 1dedd4b6278..c967fbea5ab 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreFateIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test.fate.user; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.fate.user.UserFateStore.getRowId; import static org.apache.accumulo.test.fate.user.UserFateStoreIT.createFateTable; @@ -56,9 +57,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { createFateTable(client, table); - testMethod.execute( - new UserFateStore<>(client, table, null, null, maxDeferred, fateIdGenerator), - getCluster().getServerContext()); + testMethod.execute(new UserFateStore<>(client, table, createDummyLockID(), null, maxDeferred, + fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java index 979038f7b82..07b003426e9 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/user/UserFateStoreIT.java @@ -86,7 +86,7 @@ private static class TestUserFateStore extends UserFateStore { // use the list of fateIds to simulate collisions on fateIds public TestUserFateStore(ClientContext context, String tableName, List fateIds) { - super(context, tableName, null, null); + super(context, tableName, createDummyLockID(), null); this.fateIdIterator = fateIds.iterator(); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index 13567829c16..bb941868a07 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -21,6 +21,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -262,11 +263,11 @@ public void getFateStatus() { InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - MetaFateStore mfs = - new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, null, null); + MetaFateStore mfs = new MetaFateStore<>( + ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createDummyLockID(), null); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); - UserFateStore ufs = new UserFateStore<>(context, null, null); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); @@ -356,8 +357,8 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - MetaFateStore mfs = - new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, null, null); + MetaFateStore mfs = new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, + zk, createDummyLockID(), null); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); AdminUtil.FateStatus fateStatus = admin.getStatus(mfs, zk, lockPath, null, null, null); @@ -387,7 +388,7 @@ private boolean lookupFateInAccumulo(final String tableName) throws KeeperExcept log.trace("tid: {}", tableId); - UserFateStore ufs = new UserFateStore<>(context, null, null); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); AdminUtil.FateStatus fateStatus = admin.getStatus(ufs, null, null, null); log.trace("current fates: {}", fateStatus.getTransactions().size()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java index 6556a721d7d..001bc0033be 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java @@ -19,6 +19,7 @@ package org.apache.accumulo.test.functional; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.accumulo.core.fate.AbstractFateStore.createDummyLockID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FLUSH_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -234,9 +235,9 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); ZooReaderWriter zk = context.getZooReaderWriter(); - MetaFateStore mfs = - new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, null, null); - UserFateStore ufs = new UserFateStore<>(context, null, null); + MetaFateStore mfs = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, + zk, createDummyLockID(), null); + UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); Map> fateStores = Map.of(FateInstanceType.META, mfs, FateInstanceType.USER, ufs); var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); From 367bc7ce04b0e557f053c43581f86898d4f1c0f6 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Fri, 2 Aug 2024 15:22:09 -0400 Subject: [PATCH 13/17] Added check to FateStoreIT.testAbsent() Added check that both stores have the same expected functionality when trying to reserve a non-existent fate id --- .../main/java/org/apache/accumulo/test/fate/FateStoreIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java index a2c2159dcfb..938025435fb 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateStoreIT.java @@ -453,11 +453,12 @@ public void testAbsent() throws Exception { protected void testAbsent(FateStore store, ServerContext sctx) { // Ensure both store implementations have consistent behavior when reading a fateId that does - // not exists. + // not exist. var fateId = FateId.from(store.type(), UUID.randomUUID()); var txStore = store.read(fateId); + assertTrue(store.tryReserve(fateId).isEmpty()); assertEquals(TStatus.UNKNOWN, txStore.getStatus()); assertNull(txStore.top()); assertNull(txStore.getTransactionInfo(TxInfo.TX_NAME)); From 7df2f825d6829ac6600da31da7ba07f6026a77ae Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Fri, 2 Aug 2024 16:41:08 -0400 Subject: [PATCH 14/17] Trivial change to RowFateStatusFilter --- .../apache/accumulo/core/fate/user/RowFateStatusFilter.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java b/core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java index fd4459837b0..61ac19eb4b0 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/user/RowFateStatusFilter.java @@ -59,8 +59,7 @@ public void init(SortedKeyValueIterator source, Map op protected boolean filter(Text currentRow, List keys, List values) { for (int i = 0; i < keys.size(); i++) { Key key = keys.get(i); - if (key.getColumnQualifier() - .equals(FateSchema.TxColumnFamily.STATUS_COLUMN.getColumnQualifier()) + if (FateSchema.TxColumnFamily.STATUS_COLUMN.hasColumns(key) && valuesToAccept.contains(ReadOnlyFateStore.TStatus.valueOf(values.get(i).toString()))) { return true; } From 71227defdc11ecd253e5dc4690df537394155bd3 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 6 Aug 2024 09:48:08 -0400 Subject: [PATCH 15/17] Verify err msg contains fate id in MultipleStoresIT --- .../org/apache/accumulo/test/fate/MultipleStoresIT.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index 221e5f597fb..ec42ea814b9 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -198,7 +198,8 @@ private void testReserveNonExistentTxn(FateInstanceType storeType) throws Except store = new MetaFateStore<>(FATE_DIR, zk, lock, null); } - assertThrows(IllegalStateException.class, () -> store.reserve(fakeFateId)); + var err = assertThrows(IllegalStateException.class, () -> store.reserve(fakeFateId)); + assertTrue(err.getMessage().contains(fakeFateId.canonical())); } @Test @@ -301,7 +302,8 @@ private void testReserveAfterUnreserveAndReserveAfterDeleted(FateInstanceType st // Verify that the tx is now unknown since it has been deleted assertEquals(ReadOnlyFateStore.TStatus.UNKNOWN, store.read(fateId).getStatus()); // Attempt to reserve a deleted txn, should throw an exception and not wait indefinitely - assertThrows(IllegalStateException.class, () -> store.reserve(fateId)); + var err = assertThrows(IllegalStateException.class, () -> store.reserve(fateId)); + assertTrue(err.getMessage().contains(fateId.canonical())); } } From 9d46daa09ba6178ecfcd88c38f1b3d1fb8eebdb9 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Wed, 7 Aug 2024 12:31:22 -0400 Subject: [PATCH 16/17] Removed TODOs for this PR: 'TODO 4131' --- .../accumulo/core/fate/AbstractFateStore.java | 15 ++------------- .../java/org/apache/accumulo/core/fate/Fate.java | 6 ++---- .../org/apache/accumulo/core/fate/FateKey.java | 3 --- .../org/apache/accumulo/core/fate/FateStore.java | 2 +- .../apache/accumulo/core/fate/MetaFateStore.java | 1 - .../accumulo/test/fate/MultipleStoresIT.java | 4 ---- 6 files changed, 5 insertions(+), 26 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 257b724b2d0..92031a8f0f9 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -56,10 +56,6 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -// TODO 4131 should probably add support to AbstractFateStore, MetaFateStore, -// and UserFateStore to accept null lockID (maybe make this field Optional<>). -// This could replace the current createDummyLockID(). This support -// is needed since MFS and UFS aren't always created in the context of a Manager. public abstract class AbstractFateStore implements FateStore { private static final Logger log = LoggerFactory.getLogger(AbstractFateStore.class); @@ -323,13 +319,6 @@ public TStatus waitForStatusChange(EnumSet expected) { int currNumCallers = concurrentStatusChangeCallers.incrementAndGet(); try { - // TODO 4131 - // TODO make the max time a function of the number of concurrent callers, as the number of - // concurrent callers increases then increase the max wait time - // TODO could support signaling within this instance for known events - // TODO made the maxWait low so this would be responsive... that may put a lot of load in - // the case there are lots of things waiting... - // Made maxWait = num of curr callers var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25)) .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(currNumCallers)) .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry(); @@ -447,8 +436,8 @@ protected Serializable deserializeTxInfo(TxInfo txInfo, byte[] data) { } /** - * TODO 4131 this is a temporary method used to create a dummy lock when using a FateStore outside - * of the context of a Manager (one example is testing). + * this is a temporary method used to create a dummy lock when using a FateStore outside the + * context of a Manager (one example is testing) so reservations can still be made. * * @return a dummy {@link ZooUtil.LockID} */ diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index 21ca036cd23..b7cbc35bc06 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -379,12 +379,10 @@ public Fate(T environment, FateStore store, boolean runDeadResCleaner, ScheduledExecutorService deadResCleanerExecutor = null; if (runDeadResCleaner) { - // Create a dead reservation cleaner for this store that will periodically (every 30 seconds) - // clean up reservations held by dead processes, if they exist. + // Create a dead reservation cleaner for this store that will periodically clean up + // reservations held by dead processes, if they exist. deadResCleanerExecutor = ThreadPools.getServerThreadPools().createScheduledExecutorService(1, store.type() + "-dead-reservation-cleaner-pool"); - // TODO 4131 periodic cleanup runs every 30 seconds - // Should this be longer? Shorter? A configurable Property? A function of something? ScheduledFuture deadReservationCleaner = deadResCleanerExecutor .scheduleWithFixedDelay(new DeadReservationCleaner(), 3, 30, SECONDS); ThreadPools.watchCriticalScheduledTask(deadReservationCleaner); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java b/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java index 210621b24c5..6c1663627c7 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java @@ -32,9 +32,6 @@ public class FateKey { - // TODO 4131 noticed FateKey is logged, but doesn't have a toString() - // a toString() method should be added. - private final FateKeyType type; private final Optional keyExtent; private final Optional compactionId; diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 38dda29e645..6b7b68baf58 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -126,7 +126,7 @@ class FateReservation { // The LockID (provided by the Manager running the FATE which uses this store) which is used for // identifying dead Managers, so their reservations can be deleted and picked up again since // they can no longer be worked on. - private final ZooUtil.LockID lockID; // TODO 4131 not sure if this is the best type for this + private final ZooUtil.LockID lockID; // The UUID generated on a reservation attempt (tryReserve()) used to uniquely identify that // attempt. This is useful for the edge case where the reservation is sent to the server // (Tablet Server for UserFateStore and the ZooKeeper Server for MetaFateStore), but the server diff --git a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java index 69b9bab0818..d801167d074 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/MetaFateStore.java @@ -59,7 +59,6 @@ //TODO use zoocache? - ACCUMULO-1297 //TODO handle zookeeper being down gracefully - ACCUMULO-1297 -// TODO 4131 noticed this class is not in the fate.zookeeper package. Should it be? public class MetaFateStore extends AbstractFateStore { private static final Logger log = LoggerFactory.getLogger(MetaFateStore.class); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java index ec42ea814b9..57070bacde1 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/MultipleStoresIT.java @@ -66,10 +66,6 @@ import com.google.common.collect.Sets; -// TODO 4131 could potentially have separate classes for testing MetaFateStore and UserFateStore -// similar to how FateTestRunner is used, however that interface doesn't work as nicely here -// since we are using multiple stores instead of just one. Can do something similar to -// FateTestRunner here if desired public class MultipleStoresIT extends SharedMiniClusterBase { private static final Logger LOG = LoggerFactory.getLogger(MultipleStoresIT.class); From 35572521b0004d868a6a91eb4cb9d39174c39e38 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Wed, 21 Aug 2024 13:45:47 -0400 Subject: [PATCH 17/17] removed synchronization block: - AbstractFateStore still had a synchronization block for accessing `deferred` which was changed to a synchronized map in this PR; removed since there shouldn't be any more sync blocks in this class. --- .../apache/accumulo/core/fate/AbstractFateStore.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 7d368df23b7..96d5805e6e9 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -193,12 +193,10 @@ public void runnable(AtomicBoolean keepWaiting, Consumer idConsumer) { if (seen.get() == 0) { if (beforeCount == unreservedRunnableCount.getCount()) { long waitTime = 5000; - synchronized (AbstractFateStore.this) { - if (!deferred.isEmpty()) { - waitTime = deferred.values().stream() - .mapToLong(countDownTimer -> countDownTimer.timeLeft(TimeUnit.MILLISECONDS)).min() - .getAsLong(); - } + if (!deferred.isEmpty()) { + waitTime = deferred.values().stream() + .mapToLong(countDownTimer -> countDownTimer.timeLeft(TimeUnit.MILLISECONDS)).min() + .getAsLong(); } if (waitTime > 0) {