From 87a3afa982296b1ee3f3f29c4558894ce770833c Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Mon, 20 Nov 2023 18:43:41 -0500 Subject: [PATCH 01/12] WIP distributed FATE Adds support to FATE for running multiple instances in different processes. Three major changes were made to support this. First, reserving a fate operation was moved from memory to zookeeper. Second, support for partitioning operations between multiple FATE instances was added. Partitioning is accomplished via a new class at the manager lever named PartitionData. This class provides the total number of manager processes and which unique number a given process is. Third, a new thread was added to fate instances that has the sole job of finding work. This change supports multiple fate instances, but does not make any changes to the manager to make this an actuality. The manager would need broader changes to support to multiple manager processes. When making these changes FATE can simply be passed a PartitionData supplier. This change is one part of a larger set of changes to support running multiple manager processes. Extensive changes were made to the internal interface used to store FATE data. These changes were required to move reserving data from in memory data structures to zookeeper. The interface was modified to return a new per fate operation type. On the old interface, one used to continually pass around a fate operation id. Now the following is done instead. ```java FatesStore allFates = ...; FateStore individualFateStore = FatesStore.reserve(123); individualFateStore.setState(FAILED); individualFateStore.unreserve(); ``` With the above changes to the FATE storage interface its easy to have an internal UUID associated with the reserved FATE operation, this was not possible with the old interface. This change also made it easier to handle deleting a reserved fate operation. The way operations to work on are found was changed substantially in this PR. There is a new single thread that looks for work using the partitioning information. If this thread does not find any work it will start doing exponential backoff. This changes was made for two reasons. First to avoid all threads scanning the FATE data store looking for work. Second now that there are multiple processes adding and running FATE op, a process can not rely on in memory signals to know to look for work. So having a single thread poll with exponential backoff is an efficient way to find work in this new scenario. If there a 10 fate instances each with 32 thread, this avoid having 320 threads scanning the store looking for work. Instead only 10 threads would be scanning in this scenario. Eventually when fate supports storing data in an Accumulo table, this scanning can use range partitioning making it very efficient. The scanning done in the zookeeper store in this PR uses hash partitioning, so each fate instance scans all data and filters out things outside of its partition. To get an overview of the functionality these changes offer, look at the new MultipleFateInstancesIT.testMultipleFateInstances() test. Fates ageoff functionality was dropped in this PR. The functionality is still needed, however the way it was implemented would not work well for multiple instances of FATE. Rather than attempt to change something that needed to reimplemented, I just removed it for now. Think it will be cleaner to add a new implementation later. Future work * Multiple managers that start fate and pass in partition data * Rework admin tools to provide information about locations where fate ops have run and are running * Billion splits test. When we can have multiple managers with 1000s of total fate threads, run a test on small cluster to create a table with 1 billion+ tablets. Would probably also need FATE data stored in an Accumulo table for this test. * Re-implement the age off functionality in such a way that it does not store data in memory * Implement finding FATE ops that are reserved by dead processes and remove the reservation * Fate metrics and upgrade code may have been broken by these changes, may fix that in this PR * Maybe other functional services running in the manager like TGW can use the PartitionData class to distribute work. * Within FATE operations, singleton manager services are used like the event coordinator. This will need to be addressed to support multiple managers. * The thread looking for work should consider the minimum deferment time when deciding how long to sleep the case where nothing was found. --- .../apache/accumulo/core/fate/AdminUtil.java | 69 +- .../accumulo/core/fate/AgeOffStore.java | 247 ------- .../org/apache/accumulo/core/fate/Fate.java | 265 ++++++-- .../apache/accumulo/core/fate/FatesStore.java | 111 +++ .../core/fate/ReadOnlyFatesStore.java | 128 ++++ .../accumulo/core/fate/ReadOnlyTStore.java | 144 ---- .../org/apache/accumulo/core/fate/TStore.java | 88 --- .../accumulo/core/fate/ZooFatesStore.java | 635 ++++++++++++++++++ .../apache/accumulo/core/fate/ZooStore.java | 574 ---------------- .../accumulo/core/fate/zookeeper/ZooUtil.java | 1 - .../accumulo/core/logging/FateLogger.java | 190 +++--- .../accumulo/core/manager/PartitionData.java | 92 +++ .../accumulo/core/fate/AgeOffStoreTest.java | 163 ----- .../apache/accumulo/core/fate/TestStore.java | 107 --- .../apache/accumulo/server/util/Admin.java | 21 +- .../util/fateCommand/FateSummaryReport.java | 4 +- .../util/fateCommand/SummaryReportTest.java | 4 +- .../util/fateCommand/TxnDetailsTest.java | 8 +- .../accumulo/manager/FateServiceHandler.java | 6 +- .../org/apache/accumulo/manager/Manager.java | 22 +- .../metrics/fate/FateMetricValues.java | 10 +- .../manager/metrics/fate/FateMetrics.java | 30 +- .../manager/upgrade/UpgradeCoordinator.java | 24 +- .../accumulo/test/fate/zookeeper/FateIT.java | 71 +- .../zookeeper/MultipleFateInstancesIT.java | 246 +++++++ .../test/functional/FateConcurrencyIT.java | 12 +- .../test/functional/FunctionalTestUtils.java | 5 +- 27 files changed, 1676 insertions(+), 1601 deletions(-) delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/TStore.java create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java create mode 100644 core/src/main/java/org/apache/accumulo/core/manager/PartitionData.java delete mode 100644 core/src/test/java/org/apache/accumulo/core/fate/AgeOffStoreTest.java delete mode 100644 core/src/test/java/org/apache/accumulo/core/fate/TestStore.java create mode 100644 test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index 0601f4a8c39..369049de5b3 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -33,7 +33,6 @@ import java.util.Map.Entry; import java.util.Set; -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; import org.apache.accumulo.core.fate.zookeeper.FateLock; import org.apache.accumulo.core.fate.zookeeper.FateLock.FateLockPath; import org.apache.accumulo.core.fate.zookeeper.ZooReader; @@ -71,15 +70,15 @@ public AdminUtil(boolean exitOnError) { public static class TransactionStatus { private final long txid; - private final TStatus status; + private final ReadOnlyFatesStore.FateStatus status; private final String txName; private final List hlocks; private final List wlocks; private final String top; private final long timeCreated; - private TransactionStatus(Long tid, TStatus status, String txName, List hlocks, - List wlocks, String top, Long timeCreated) { + private TransactionStatus(Long tid, ReadOnlyFatesStore.FateStatus status, String txName, + List hlocks, List wlocks, String top, Long timeCreated) { this.txid = tid; this.status = status; @@ -99,7 +98,7 @@ public String getTxid() { return FastFormat.toHexString(txid); } - public TStatus getStatus() { + public ReadOnlyFatesStore.FateStatus getStatus() { return status; } @@ -210,15 +209,15 @@ public Map> getDanglingWaitingLocks() { /** * Returns a list of the FATE transactions, optionally filtered by transaction id and status. This * method does not process lock information, if lock information is desired, use - * {@link #getStatus(ReadOnlyTStore, ZooReader, ServiceLockPath, Set, EnumSet)} + * {@link #getStatus(ReadOnlyFatesStore, ZooReader, ServiceLockPath, Set, EnumSet)} * * @param zs read-only zoostore * @param filterTxid filter results to include for provided transaction ids. * @param filterStatus filter results to include only provided status types * @return list of FATE transactions that match filter criteria */ - public List getTransactionStatus(ReadOnlyTStore zs, Set filterTxid, - EnumSet filterStatus) { + public List getTransactionStatus(ReadOnlyFatesStore zs, + Set filterTxid, EnumSet filterStatus) { FateStatus status = getTransactionStatus(zs, filterTxid, filterStatus, Collections.>emptyMap(), Collections.>emptyMap()); @@ -239,8 +238,9 @@ public List getTransactionStatus(ReadOnlyTStore zs, Set zs, ZooReader zk, - ServiceLock.ServiceLockPath lockPath, Set filterTxid, EnumSet filterStatus) + public FateStatus getStatus(ReadOnlyFatesStore zs, ZooReader zk, + ServiceLock.ServiceLockPath lockPath, Set filterTxid, + EnumSet filterStatus) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); Map> waitingLocks = new HashMap<>(); @@ -332,8 +332,8 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, * @param waitingLocks populated list of locks held by transaction - or an empty map if none. * @return current fate and lock status */ - private FateStatus getTransactionStatus(ReadOnlyTStore zs, Set filterTxid, - EnumSet filterStatus, Map> heldLocks, + private FateStatus getTransactionStatus(ReadOnlyFatesStore zs, Set filterTxid, + EnumSet filterStatus, Map> heldLocks, Map> waitingLocks) { List transactions = zs.list(); @@ -341,9 +341,9 @@ private FateStatus getTransactionStatus(ReadOnlyTStore zs, Set filterTx for (Long tid : transactions) { - zs.reserve(tid); + var opStore = zs.read(tid); - String txName = (String) zs.getTransactionInfo(tid, Fate.TxInfo.TX_NAME); + String txName = (String) opStore.getTransactionInfo(Fate.TxInfo.TX_NAME); List hlocks = heldLocks.remove(tid); @@ -358,16 +358,14 @@ private FateStatus getTransactionStatus(ReadOnlyTStore zs, Set filterTx } String top = null; - ReadOnlyRepo repo = zs.top(tid); + ReadOnlyRepo repo = opStore.top(); if (repo != null) { top = repo.getName(); } - TStatus status = zs.getStatus(tid); + ReadOnlyFatesStore.FateStatus status = opStore.getStatus(); - long timeCreated = zs.timeCreated(tid); - - zs.unreserve(tid, 0); + long timeCreated = opStore.timeCreated(); if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) { statuses.add(new TransactionStatus(tid, status, txName, hlocks, wlocks, top, timeCreated)); @@ -378,7 +376,8 @@ private FateStatus getTransactionStatus(ReadOnlyTStore zs, Set filterTx } - private boolean includeByStatus(TStatus status, EnumSet filterStatus) { + private boolean includeByStatus(ReadOnlyFatesStore.FateStatus status, + EnumSet filterStatus) { return (filterStatus == null) || filterStatus.contains(status); } @@ -386,13 +385,14 @@ private boolean includeByTxid(Long tid, Set filterTxid) { return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid); } - public void printAll(ReadOnlyTStore zs, ZooReader zk, + public void printAll(ReadOnlyFatesStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath) throws KeeperException, InterruptedException { print(zs, zk, tableLocksPath, new Formatter(System.out), null, null); } - public void print(ReadOnlyTStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath, - Formatter fmt, Set filterTxid, EnumSet filterStatus) + public void print(ReadOnlyFatesStore zs, ZooReader zk, + ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set filterTxid, + EnumSet filterStatus) throws KeeperException, InterruptedException { FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus); @@ -417,8 +417,10 @@ public void print(ReadOnlyTStore zs, ZooReader zk, ServiceLock.ServiceLockPat } } - public boolean prepDelete(TStore zs, ZooReaderWriter zk, ServiceLockPath path, + public boolean prepDelete(FatesStore zs, ZooReaderWriter zk, ServiceLockPath path, String txidStr) { + // TODO do not need global lock now + // TODO need way to see what process holds a reservation if (!checkGlobalLock(zk, path)) { return false; } @@ -431,8 +433,9 @@ public boolean prepDelete(TStore zs, ZooReaderWriter zk, ServiceLockPath path return false; } boolean state = false; - zs.reserve(txid); - TStatus ts = zs.getStatus(txid); + + var opStore = zs.reserve(txid); + ReadOnlyFatesStore.FateStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -445,16 +448,16 @@ public boolean prepDelete(TStore zs, ZooReaderWriter zk, ServiceLockPath path case FAILED_IN_PROGRESS: case SUCCESSFUL: System.out.printf("Deleting transaction: %016x (%s)%n", txid, ts); - zs.delete(txid); + opStore.delete(); state = true; break; } - zs.unreserve(txid, 0); + opStore.unreserve(0); return state; } - public boolean prepFail(TStore zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath, + public boolean prepFail(FatesStore zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath, String txidStr) { if (!checkGlobalLock(zk, zLockManagerPath)) { return false; @@ -468,8 +471,8 @@ public boolean prepFail(TStore zs, ZooReaderWriter zk, ServiceLockPath zLockM return false; } boolean state = false; - zs.reserve(txid); - TStatus ts = zs.getStatus(txid); + var opStore = zs.reserve(txid); + ReadOnlyFatesStore.FateStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -479,7 +482,7 @@ public boolean prepFail(TStore zs, ZooReaderWriter zk, ServiceLockPath zLockM case IN_PROGRESS: case NEW: System.out.printf("Failing transaction: %016x (%s)%n", txid, ts); - zs.setStatus(txid, TStatus.FAILED_IN_PROGRESS); + opStore.setStatus(ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS); state = true; break; @@ -494,7 +497,7 @@ public boolean prepFail(TStore zs, ZooReaderWriter zk, ServiceLockPath zLockM break; } - zs.unreserve(txid, 0); + opStore.unreserve(0); return state; } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java deleted file mode 100644 index 5ed59f21fe5..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java +++ /dev/null @@ -1,247 +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; - -import java.io.Serializable; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This store removes Repos, in the store it wraps, that are in a finished or new state for more - * than a configurable time period. - * - * No external time source is used. It starts tracking idle time when its created. - */ -public class AgeOffStore implements TStore { - - public interface TimeSource { - long currentTimeMillis(); - } - - private static final Logger log = LoggerFactory.getLogger(AgeOffStore.class); - - private final ZooStore store; - private Map candidates; - private long ageOffTime; - private long minTime; - private TimeSource timeSource; - - private synchronized void updateMinTime() { - minTime = Long.MAX_VALUE; - - for (Long time : candidates.values()) { - if (time < minTime) { - minTime = time; - } - } - } - - private synchronized void addCandidate(long txid) { - long time = timeSource.currentTimeMillis(); - candidates.put(txid, time); - if (time < minTime) { - minTime = time; - } - } - - private synchronized void removeCandidate(long txid) { - Long time = candidates.remove(txid); - if (time != null && time <= minTime) { - updateMinTime(); - } - } - - public void ageOff() { - HashSet oldTxs = new HashSet<>(); - - synchronized (this) { - long time = timeSource.currentTimeMillis(); - if (minTime < time && time - minTime >= ageOffTime) { - for (Entry entry : candidates.entrySet()) { - if (time - entry.getValue() >= ageOffTime) { - oldTxs.add(entry.getKey()); - } - } - - candidates.keySet().removeAll(oldTxs); - updateMinTime(); - } - } - - for (Long txid : oldTxs) { - try { - store.reserve(txid); - try { - switch (store.getStatus(txid)) { - case NEW: - case FAILED: - case SUCCESSFUL: - store.delete(txid); - log.debug("Aged off FATE tx {}", FateTxId.formatTid(txid)); - break; - default: - break; - } - - } finally { - store.unreserve(txid, 0); - } - } catch (Exception e) { - log.warn("Failed to age off FATE tx " + FateTxId.formatTid(txid), e); - } - } - } - - public AgeOffStore(ZooStore store, long ageOffTime, TimeSource timeSource) { - this.store = store; - this.ageOffTime = ageOffTime; - this.timeSource = timeSource; - candidates = new HashMap<>(); - - minTime = Long.MAX_VALUE; - - List txids = store.list(); - for (Long txid : txids) { - store.reserve(txid); - try { - switch (store.getStatus(txid)) { - case NEW: - case FAILED: - case SUCCESSFUL: - addCandidate(txid); - break; - default: - break; - } - } finally { - store.unreserve(txid, 0); - } - } - } - - @Override - public long create() { - long txid = store.create(); - addCandidate(txid); - return txid; - } - - @Override - public long reserve() { - return store.reserve(); - } - - @Override - public void reserve(long tid) { - store.reserve(tid); - } - - @Override - public boolean tryReserve(long tid) { - return store.tryReserve(tid); - } - - @Override - public void unreserve(long tid, long deferTime) { - store.unreserve(tid, deferTime); - } - - @Override - public Repo top(long tid) { - return store.top(tid); - } - - @Override - public void push(long tid, Repo repo) throws StackOverflowException { - store.push(tid, repo); - } - - @Override - public void pop(long tid) { - store.pop(tid); - } - - @Override - public org.apache.accumulo.core.fate.TStore.TStatus getStatus(long tid) { - return store.getStatus(tid); - } - - @Override - public void setStatus(long tid, org.apache.accumulo.core.fate.TStore.TStatus status) { - store.setStatus(tid, status); - - switch (status) { - case SUBMITTED: - case IN_PROGRESS: - case FAILED_IN_PROGRESS: - removeCandidate(tid); - break; - case FAILED: - case SUCCESSFUL: - addCandidate(tid); - break; - default: - break; - } - } - - @Override - public org.apache.accumulo.core.fate.TStore.TStatus waitForStatusChange(long tid, - EnumSet expected) { - return store.waitForStatusChange(tid, expected); - } - - @Override - public void setTransactionInfo(long tid, Fate.TxInfo txInfo, Serializable val) { - store.setTransactionInfo(tid, txInfo, val); - } - - @Override - public Serializable getTransactionInfo(long tid, Fate.TxInfo txInfo) { - return store.getTransactionInfo(tid, txInfo); - } - - @Override - public void delete(long tid) { - store.delete(tid); - removeCandidate(tid); - } - - @Override - public List list() { - return store.list(); - } - - @Override - public long timeCreated(long tid) { - return store.timeCreated(tid); - } - - @Override - public List> getStack(long tid) { - return store.getStack(tid); - } -} 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 1f0ae0d7583..fe018f7a33e 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 @@ -19,33 +19,41 @@ package org.apache.accumulo.core.fate; import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED_IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.NEW; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.SUCCESSFUL; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.UNKNOWN; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.NEW; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUCCESSFUL; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.UNKNOWN; import static org.apache.accumulo.core.util.ShutdownUtil.isIOException; import java.util.EnumSet; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.function.Supplier; import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus; import org.apache.accumulo.core.logging.FateLogger; +import org.apache.accumulo.core.manager.PartitionData; +import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.ShutdownUtil; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.core.util.threads.Threads; import org.apache.thrift.TApplicationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,36 +66,140 @@ public class Fate { private static final Logger log = LoggerFactory.getLogger(Fate.class); private final Logger runnerLog = LoggerFactory.getLogger(TransactionRunner.class); - private final TStore store; + private final FatesStore store; private final T environment; private final ScheduledThreadPoolExecutor fatePoolWatcher; private final ExecutorService executor; - private static final EnumSet FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); + private static final EnumSet FINISHED_STATES = + EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); private final AtomicBoolean keepRunning = new AtomicBoolean(true); + private final Supplier partitionDataSupplier; + private final BlockingQueue workQueue; + private final Thread workFinder; public enum TxInfo { TX_NAME, AUTO_CLEAN, EXCEPTION, RETURN_VALUE } + // TODO add a task that periodically looks for fate task reserved by dead instances, only run in + // partition zero + private class LockCleaner implements Runnable { + public void run() { + while (keepRunning.get()) { + var partitionData = partitionDataSupplier.get(); + if (partitionData.shouldRun(PartitionData.SingletonManagerService.FATE_LOCK_CLEANUP)) { + // run cleanup + } + // sleep a long time + } + } + } + + /** + * A single thread that finds transactions to work on and queues them up. Do not want each worker + * thread going to the store and looking for work as it would place more load on the store. + */ + private class WorkFinder implements Runnable { + + private Retry newRetry() { + // ELASTICITY_TODO the max time to retry may be store depependent, depends on how expensive + // the read is in the impl. The more expensive the read is the high we may want to make the + // max. Can figure this out after we have an Accumulo table store. + return Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS) + .incrementBy(25, MILLISECONDS).maxWait(10, SECONDS).backOffFactor(1.5) + .logInterval(3, MINUTES).createRetry(); + } + + public void run() { + + try { + Retry retry = newRetry(); + PartitionData lastPartitionData = partitionDataSupplier.get(); + + while (keepRunning.get()) { + var partitionData = partitionDataSupplier.get(); + if (!partitionData.equals(lastPartitionData)) { + // Partition data changed, so lets clear anything queued to avoid unnecessary + // contention. + // The queue does not have to be cleared here for overall correctness, this is just an + // attempt to avoid some unnecessary work caused by multiple processes trying to reserve + // the same fate transaction. + workQueue.clear(); + } + lastPartitionData = partitionData; + + var iter = store.runnable(partitionData); + + int count = 0; + while (iter.hasNext() && keepRunning.get()) { + Long txid = iter.next(); + count++; + try { + while (keepRunning.get()) { + if (workQueue.offer(txid, 100, TimeUnit.MILLISECONDS)) { + break; + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + + if (count == 0) { + // When nothing was found in the store, then start doing exponential backoff before + // reading from the store again in order to avoid overwhelming the store. + try { + retry.waitForNextAttempt(log, + "Find work for fate partition " + partitionData.toString()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } else { + // reset the retry object + retry.logCompletion(log, "Find work for fate partition " + partitionData.toString()); + retry = newRetry(); + } + } + } catch (Exception e) { + if (keepRunning.get()) { + log.warn("Failure while attempting to find work for fate", e); + } else { + log.debug("Failure while attempting to find work for fate", e); + } + } + } + } + private class TransactionRunner implements Runnable { @Override public void run() { while (keepRunning.get()) { long deferTime = 0; - Long tid = null; + FatesStore.FateStore opStore = null; try { - tid = store.reserve(); - TStatus status = store.getStatus(tid); - Repo op = store.top(tid); + var unreservedTid = workQueue.poll(100, MILLISECONDS); + if (unreservedTid == null) { + continue; + } + var optionalopStore = store.tryReserve(unreservedTid); + if (optionalopStore.isPresent()) { + opStore = optionalopStore.orElseThrow(); + } else { + continue; + } + FateStatus status = opStore.getStatus(); + Repo op = opStore.top(); if (status == FAILED_IN_PROGRESS) { - processFailed(tid, op); + processFailed(opStore, op); } else { Repo prevOp = null; try { - deferTime = op.isReady(tid, environment); + deferTime = op.isReady(opStore.getID(), environment); // Here, deferTime is only used to determine success (zero) or failure (non-zero), // proceeding on success and returning to the while loop on failure. @@ -95,16 +207,16 @@ public void run() { if (deferTime == 0) { prevOp = op; if (status == SUBMITTED) { - store.setStatus(tid, IN_PROGRESS); + opStore.setStatus(IN_PROGRESS); } - op = op.call(tid, environment); + op = op.call(opStore.getID(), environment); } else { continue; } } catch (Exception e) { - blockIfHadoopShutdown(tid, e); - transitionToFailed(tid, e); + blockIfHadoopShutdown(opStore.getID(), e); + transitionToFailed(opStore, e); continue; } @@ -112,18 +224,18 @@ public void run() { // transaction is finished String ret = prevOp.getReturn(); if (ret != null) { - store.setTransactionInfo(tid, TxInfo.RETURN_VALUE, ret); + opStore.setTransactionInfo(TxInfo.RETURN_VALUE, ret); } - store.setStatus(tid, SUCCESSFUL); - doCleanUp(tid); + opStore.setStatus(SUCCESSFUL); + doCleanUp(opStore); } else { try { - store.push(tid, op); + opStore.push(op); } catch (StackOverflowException e) { // the op that failed to push onto the stack was never executed, so no need to undo // it // just transition to failed and undo the ops that executed - transitionToFailed(tid, e); + transitionToFailed(opStore, e); continue; } } @@ -131,8 +243,8 @@ public void run() { } catch (Exception e) { runnerLog.error("Uncaught exception in FATE runner thread.", e); } finally { - if (tid != null) { - store.unreserve(tid, deferTime); + if (opStore != null) { + opStore.unreserve(deferTime); } } } @@ -166,8 +278,8 @@ private void blockIfHadoopShutdown(long tid, Exception e) { } } - private void transitionToFailed(long tid, Exception e) { - String tidStr = FateTxId.formatTid(tid); + private void transitionToFailed(FatesStore.FateStore opStore, Exception e) { + String tidStr = FateTxId.formatTid(opStore.getID()); final String msg = "Failed to execute Repo " + tidStr; // Certain FATE ops that throw exceptions don't need to be propagated up to the Monitor // as a warning. They're a normal, handled failure condition. @@ -178,32 +290,32 @@ private void transitionToFailed(long tid, Exception e) { } else { log.warn(msg, e); } - store.setTransactionInfo(tid, TxInfo.EXCEPTION, e); - store.setStatus(tid, FAILED_IN_PROGRESS); + opStore.setTransactionInfo(TxInfo.EXCEPTION, e); + opStore.setStatus(FAILED_IN_PROGRESS); log.info("Updated status for Repo with {} to FAILED_IN_PROGRESS", tidStr); } - private void processFailed(long tid, Repo op) { + private void processFailed(FatesStore.FateStore opStore, Repo op) { while (op != null) { - undo(tid, op); + undo(opStore.getID(), op); - store.pop(tid); - op = store.top(tid); + opStore.pop(); + op = opStore.top(); } - store.setStatus(tid, FAILED); - doCleanUp(tid); + opStore.setStatus(FAILED); + doCleanUp(opStore); } - private void doCleanUp(long tid) { - Boolean autoClean = (Boolean) store.getTransactionInfo(tid, TxInfo.AUTO_CLEAN); + private void doCleanUp(FatesStore.FateStore opStore) { + Boolean autoClean = (Boolean) opStore.getTransactionInfo(TxInfo.AUTO_CLEAN); if (autoClean != null && autoClean) { - store.delete(tid); + opStore.delete(); } else { // no longer need persisted operations, so delete them to save space in case // TX is never cleaned up... - while (store.top(tid) != null) { - store.pop(tid); + while (opStore.top() != null) { + opStore.pop(); } } } @@ -223,12 +335,16 @@ private void undo(long tid, Repo op) { * * @param toLogStrFunc A function that converts Repo to Strings that are suitable for logging */ - public Fate(T environment, TStore store, Function,String> toLogStrFunc, - AccumuloConfiguration conf) { + public Fate(T environment, FatesStore store, Function,String> toLogStrFunc, + Supplier partitionDataSupplier, AccumuloConfiguration conf) { this.store = FateLogger.wrap(store, toLogStrFunc); this.environment = environment; + this.partitionDataSupplier = partitionDataSupplier; final ThreadPoolExecutor pool = ThreadPools.getServerThreadPools().createExecutorService(conf, Property.MANAGER_FATE_THREADPOOL_SIZE, true); + // TODO this queue does not resize when config changes + this.workQueue = + new ArrayBlockingQueue<>(conf.getCount(Property.MANAGER_FATE_THREADPOOL_SIZE) * 4); this.fatePoolWatcher = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf); ThreadPools.watchCriticalScheduledTask(fatePoolWatcher.schedule(() -> { @@ -255,6 +371,9 @@ public Fate(T environment, TStore store, Function,String> toLogStrFun } }, 3, SECONDS)); this.executor = pool; + + this.workFinder = Threads.createThread("Fate work finder", new WorkFinder()); + this.workFinder.start(); } // get a transaction id back to the requester before doing any work @@ -266,13 +385,13 @@ public long startTransaction() { // multiple times for a transaction... but it will only seed once public void seedTransaction(String txName, long tid, Repo repo, boolean autoCleanUp, String goalMessage) { - store.reserve(tid); + var opStore = store.reserve(tid); try { - if (store.getStatus(tid) == NEW) { - if (store.top(tid) == null) { + if (opStore.getStatus() == NEW) { + if (opStore.top() == null) { try { log.info("Seeding {} {}", FateTxId.formatTid(tid), goalMessage); - store.push(tid, repo); + opStore.push(repo); } catch (StackOverflowException e) { // this should not happen throw new IllegalStateException(e); @@ -280,22 +399,22 @@ public void seedTransaction(String txName, long tid, Repo repo, boolean autoC } if (autoCleanUp) { - store.setTransactionInfo(tid, TxInfo.AUTO_CLEAN, autoCleanUp); + opStore.setTransactionInfo(TxInfo.AUTO_CLEAN, autoCleanUp); } - store.setTransactionInfo(tid, TxInfo.TX_NAME, txName); + opStore.setTransactionInfo(TxInfo.TX_NAME, txName); - store.setStatus(tid, SUBMITTED); + opStore.setStatus(SUBMITTED); } } finally { - store.unreserve(tid, 0); + opStore.unreserve(0); } } // check on the transaction - public TStatus waitForCompletion(long tid) { - return store.waitForStatusChange(tid, FINISHED_STATES); + public FateStatus waitForCompletion(long tid) { + return store.read(tid).waitForStatusChange(FINISHED_STATES); } /** @@ -308,14 +427,16 @@ public TStatus waitForCompletion(long tid) { public boolean cancel(long tid) { String tidStr = FateTxId.formatTid(tid); for (int retries = 0; retries < 5; retries++) { - if (store.tryReserve(tid)) { + var optionalOpStore = store.tryReserve(tid); + if (optionalOpStore.isPresent()) { + var opStore = optionalOpStore.orElseThrow(); try { - TStatus status = store.getStatus(tid); + FateStatus status = opStore.getStatus(); log.info("status is: {}", status); if (status == NEW || status == SUBMITTED) { - store.setTransactionInfo(tid, TxInfo.EXCEPTION, new TApplicationException( + opStore.setTransactionInfo(TxInfo.EXCEPTION, new TApplicationException( TApplicationException.INTERNAL_ERROR, "Fate transaction cancelled by user")); - store.setStatus(tid, FAILED_IN_PROGRESS); + opStore.setStatus(FAILED_IN_PROGRESS); log.info("Updated status for {} to FAILED_IN_PROGRESS because it was cancelled by user", tidStr); return true; @@ -324,7 +445,7 @@ public boolean cancel(long tid) { return false; } } finally { - store.unreserve(tid, 0); + opStore.unreserve(0); } } else { // reserved, lets retry. @@ -337,14 +458,15 @@ public boolean cancel(long tid) { // resource cleanup public void delete(long tid) { - store.reserve(tid); + // TODO need to handle case of not existing + var opStore = store.reserve(tid); try { - switch (store.getStatus(tid)) { + switch (opStore.getStatus()) { case NEW: case SUBMITTED: case FAILED: case SUCCESSFUL: - store.delete(tid); + opStore.delete(); break; case FAILED_IN_PROGRESS: case IN_PROGRESS: @@ -355,34 +477,34 @@ public void delete(long tid) { break; } } finally { - store.unreserve(tid, 0); + opStore.unreserve(0); } } public String getReturn(long tid) { - store.reserve(tid); + var opStore = store.reserve(tid); try { - if (store.getStatus(tid) != SUCCESSFUL) { + if (opStore.getStatus() != SUCCESSFUL) { throw new IllegalStateException("Tried to get exception when transaction " + FateTxId.formatTid(tid) + " not in successful state"); } - return (String) store.getTransactionInfo(tid, TxInfo.RETURN_VALUE); + return (String) opStore.getTransactionInfo(TxInfo.RETURN_VALUE); } finally { - store.unreserve(tid, 0); + opStore.unreserve(0); } } // get reportable failures public Exception getException(long tid) { - store.reserve(tid); + var opStore = store.reserve(tid); try { - if (store.getStatus(tid) != FAILED) { + if (opStore.getStatus() != FAILED) { throw new IllegalStateException("Tried to get exception when transaction " + FateTxId.formatTid(tid) + " not in failed state"); } - return (Exception) store.getTransactionInfo(tid, TxInfo.EXCEPTION); + return (Exception) opStore.getTransactionInfo(TxInfo.EXCEPTION); } finally { - store.unreserve(tid, 0); + opStore.unreserve(0); } } @@ -395,6 +517,7 @@ public void shutdown() { if (executor != null) { executor.shutdown(); } + workFinder.interrupt(); } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java new file mode 100644 index 00000000000..20aa969de35 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java @@ -0,0 +1,111 @@ +/* + * 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; + +import java.io.Serializable; +import java.util.Optional; + +/** + * FatesStore : a place to store fate data for all fate operations. + * + * A fate operation consists of a number of smaller idempotent operations. To use, first create a + * transaction id, and then seed the transaction with an initial operation. An executor service can + * then execute the transaction's operation, possibly pushing more operations onto the transaction + * as each step successfully completes. If a step fails, the stack can be unwound, undoing each + * operation. + */ +public interface FatesStore extends ReadOnlyFatesStore { + + /** + * Create a new transaction id + * + * @return a transaction id + */ + long create(); + + /** + * An interface that allows read/write access to the data related to a single fate operation. + */ + interface FateStore extends ReadOnlyFateStore { + + @Override + Repo top(); + + /** + * Update the given transaction with the next operation + * + * @param repo the operation + */ + void push(Repo repo) throws StackOverflowException; + + /** + * Remove the last pushed operation from the given transaction. + */ + void pop(); + + /** + * Update the state of a given transaction + * + * @param status execution status + */ + void setStatus(FateStatus status); + + /** + * Set transaction-specific information. + * + * @param txInfo name of attribute of a transaction to set. + * @param val transaction data to store + */ + void setTransactionInfo(Fate.TxInfo txInfo, Serializable val); + + /** + * Remove the transaction from the store. + */ + void delete(); + + /** + * Return the given transaction to the store. + * + * upon successful return the store now controls the referenced transaction id. caller should no + * longer interact with it. + * + * @param deferTime time in millis to keep this transaction out of the pool used in the + * {@link #reserve() reserve} method. must be non-negative. + */ + void unreserve(long deferTime); + } + + /** + * Attempt to reserve transaction + * + * @param tid transaction id + * @return true if reserved by this call, false if already reserved + */ + Optional> tryReserve(long tid); + + /** + * Reserve the specific tid. + * + * Reserving a transaction id ensures that nothing else in-process interacting via the same + * instance will be operating on that transaction id. + * + */ + FateStore reserve(long tid); + +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java new file mode 100644 index 00000000000..69f830811f4 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * 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; + +import java.io.Serializable; +import java.util.EnumSet; +import java.util.Iterator; +import java.util.List; + +import org.apache.accumulo.core.manager.PartitionData; + +/** + * Read only access to a Transaction Store. + * + * A transaction consists of a number of operations. Instances of this class may check on the queue + * of outstanding transactions but may neither modify them nor create new ones. + */ +public interface ReadOnlyFatesStore { + + /** + * Possible operational status codes. Serialized by name within stores. + */ + enum FateStatus { + /** Unseeded transaction */ + NEW, + /** Transaction that is executing */ + IN_PROGRESS, + /** Transaction has failed, and is in the process of being rolled back */ + FAILED_IN_PROGRESS, + /** Transaction has failed and has been fully rolled back */ + FAILED, + /** Transaction has succeeded */ + SUCCESSFUL, + /** Unrecognized or unknown transaction state */ + UNKNOWN, + /** Transaction that is eligible to be executed */ + SUBMITTED + } + + interface ReadOnlyFateStore { + /** + * Get the current operation for the given transaction id. + * + * Caller must have already reserved tid. + * + * @param tid transaction id, previously reserved. + * @return a read-only view of the operation + */ + ReadOnlyRepo top(); + + /** + * Get all operations on a transactions stack. Element 0 contains the most recent operation + * pushed or the top. + */ + List> getStack(); + + /** + * Get the state of a given transaction. + * + * Caller must have already reserved tid. + * + * @param tid transaction id, previously reserved. + * @return execution status + */ + FateStatus getStatus(); + + /** + * Wait for the status of a transaction to change + * + * @param tid transaction id, need not have been reserved. + * @param expected a set of possible statuses we are interested in being notified about. may not + * be null. + * @return execution status. + */ + FateStatus waitForStatusChange(EnumSet expected); + + /** + * Retrieve transaction-specific information. + * + * Caller must have already reserved tid. + * + * @param tid transaction id, previously reserved. + * @param txInfo name of attribute of a transaction to retrieve. + */ + Serializable getTransactionInfo(Fate.TxInfo txInfo); + + /** + * Retrieve the creation time of a FaTE transaction. + * + * @return creation time of transaction. + */ + long timeCreated(); + + long getID(); + } + + ReadOnlyFateStore read(long tid); + + /** + * list all transaction ids in store. + * + * @return all outstanding transactions, including those reserved by others. + */ + List list(); + + /** + * @return an iterator over fate op ids that are (IN_PROGRESS or FAILED_IN_PROGRESS) and + * unreserved. Also filter the transaction using the partitioning data so that each fate + * instance sees a different subset of all fate transactions. + */ + Iterator runnable(PartitionData partitionData); +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java deleted file mode 100644 index e4f55e4b16b..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java +++ /dev/null @@ -1,144 +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; - -import java.io.Serializable; -import java.util.EnumSet; -import java.util.List; - -/** - * Read only access to a Transaction Store. - * - * A transaction consists of a number of operations. Instances of this class may check on the queue - * of outstanding transactions but may neither modify them nor create new ones. - */ -public interface ReadOnlyTStore { - - /** - * Possible operational status codes. Serialized by name within stores. - */ - enum TStatus { - /** Unseeded transaction */ - NEW, - /** Transaction that is executing */ - IN_PROGRESS, - /** Transaction has failed, and is in the process of being rolled back */ - FAILED_IN_PROGRESS, - /** Transaction has failed and has been fully rolled back */ - FAILED, - /** Transaction has succeeded */ - SUCCESSFUL, - /** Unrecognized or unknown transaction state */ - UNKNOWN, - /** Transaction that is eligible to be executed */ - SUBMITTED - } - - /** - * Reserve a transaction that is IN_PROGRESS or FAILED_IN_PROGRESS. - * - * Reserving a transaction id ensures that nothing else in-process interacting via the same - * instance will be operating on that transaction id. - * - * @return a transaction id that is safe to interact with, chosen by the store. - */ - long reserve(); - - /** - * Reserve the specific tid. - * - * Reserving a transaction id ensures that nothing else in-process interacting via the same - * instance will be operating on that transaction id. - * - */ - void reserve(long tid); - - /** - * Return the given transaction to the store. - * - * upon successful return the store now controls the referenced transaction id. caller should no - * longer interact with it. - * - * @param tid transaction id, previously reserved. - * @param deferTime time in millis to keep this transaction out of the pool used in the - * {@link #reserve() reserve} method. must be non-negative. - */ - void unreserve(long tid, long deferTime); - - /** - * Get the current operation for the given transaction id. - * - * Caller must have already reserved tid. - * - * @param tid transaction id, previously reserved. - * @return a read-only view of the operation - */ - ReadOnlyRepo top(long tid); - - /** - * Get all operations on a transactions stack. Element 0 contains the most recent operation pushed - * or the top. - */ - List> getStack(long tid); - - /** - * Get the state of a given transaction. - * - * Caller must have already reserved tid. - * - * @param tid transaction id, previously reserved. - * @return execution status - */ - TStatus getStatus(long tid); - - /** - * Wait for the status of a transaction to change - * - * @param tid transaction id, need not have been reserved. - * @param expected a set of possible statuses we are interested in being notified about. may not - * be null. - * @return execution status. - */ - TStatus waitForStatusChange(long tid, EnumSet expected); - - /** - * Retrieve transaction-specific information. - * - * Caller must have already reserved tid. - * - * @param tid transaction id, previously reserved. - * @param txInfo name of attribute of a transaction to retrieve. - */ - Serializable getTransactionInfo(long tid, Fate.TxInfo txInfo); - - /** - * list all transaction ids in store. - * - * @return all outstanding transactions, including those reserved by others. - */ - List list(); - - /** - * Retrieve the creation time of a FaTE transaction. - * - * @param tid Transaction id, previously reserved. - * @return creation time of transaction. - */ - long timeCreated(long tid); -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/TStore.java b/core/src/main/java/org/apache/accumulo/core/fate/TStore.java deleted file mode 100644 index 8958628b7db..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/TStore.java +++ /dev/null @@ -1,88 +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; - -import java.io.Serializable; - -/** - * Transaction Store: a place to save transactions - * - * A transaction consists of a number of operations. To use, first create a transaction id, and then - * seed the transaction with an initial operation. An executor service can then execute the - * transaction's operation, possibly pushing more operations onto the transaction as each step - * successfully completes. If a step fails, the stack can be unwound, undoing each operation. - */ -public interface TStore extends ReadOnlyTStore { - - /** - * Create a new transaction id - * - * @return a transaction id - */ - long create(); - - @Override - Repo top(long tid); - - /** - * Update the given transaction with the next operation - * - * @param tid the transaction id - * @param repo the operation - */ - void push(long tid, Repo repo) throws StackOverflowException; - - /** - * Remove the last pushed operation from the given transaction. - */ - void pop(long tid); - - /** - * Update the state of a given transaction - * - * @param tid transaction id - * @param status execution status - */ - void setStatus(long tid, TStatus status); - - /** - * Set transaction-specific information. - * - * @param tid transaction id - * @param txInfo name of attribute of a transaction to set. - * @param val transaction data to store - */ - void setTransactionInfo(long tid, Fate.TxInfo txInfo, Serializable val); - - /** - * Remove the transaction from the store. - * - * @param tid the transaction id - */ - void delete(long tid); - - /** - * Attempt to reserve transaction - * - * @param tid transaction id - * @return true if reserved by this call, false if already reserved - */ - boolean tryReserve(long tid); - -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java new file mode 100644 index 00000000000..ad4be67db81 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java @@ -0,0 +1,635 @@ +/* + * 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; + +import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.accumulo.core.util.LazySingletons.RANDOM; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.UUID; + +import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; +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.manager.PartitionData; +import org.apache.accumulo.core.util.FastFormat; +import org.apache.accumulo.core.util.Retry; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.KeeperException.NodeExistsException; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + +//TODO use zoocache? - ACCUMULO-1297 +//TODO handle zookeeper being down gracefully - ACCUMULO-1297 + +public class ZooFatesStore implements FatesStore { + + private static final Logger log = LoggerFactory.getLogger(ZooFatesStore.class); + private String path; + private ZooReaderWriter zk; + + private Map defered; + + // The zookeeper lock for the process that running this store instance. + private final ZooUtil.LockID lockID; + + // TODO this was hastily written code for serialization. Should it use json? It needs better + // validation of data being serialized (like validate status, lock, and uuid). + private static class NodeValue { + final FateStatus status; + final String lock; + final String uuid; + + private NodeValue(byte[] serializedData) { + var fields = new String(serializedData, UTF_8).split(":", 3); + this.status = FateStatus.valueOf(fields[0]); + this.lock = fields[1]; + this.uuid = fields[2]; + } + + private NodeValue(FateStatus status, String lock, String uuid) { + if (lock.isEmpty() && !uuid.isEmpty() || uuid.isEmpty() && !lock.isEmpty()) { + throw new IllegalArgumentException( + "For lock and uuid expect neither is empty or both are empty. lock:'" + lock + + "' uuid:'" + uuid + "'"); + } + this.status = status; + this.lock = lock; + this.uuid = uuid; + } + + byte[] serialize() { + return (status.name() + ":" + lock + ":" + uuid).getBytes(UTF_8); + } + + public boolean isReserved() { + return !lock.isEmpty(); + } + } + + private byte[] serialize(Object o) { + + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(o); + oos.close(); + + return baos.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @SuppressFBWarnings(value = "OBJECT_DESERIALIZATION", + justification = "unsafe to store arbitrary serialized objects like this, but needed for now" + + " for backwards compatibility") + private Object deserialize(byte[] ser) { + try { + ByteArrayInputStream bais = new ByteArrayInputStream(ser); + ObjectInputStream ois = new ObjectInputStream(bais); + return ois.readObject(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (ReflectiveOperationException e) { + throw new IllegalStateException(e); + } + } + + private String getTXPath(long tid) { + return FastFormat.toHexString(path + "/tx_", tid, ""); + } + + private long parseTid(String txdir) { + return Long.parseLong(txdir.split("_")[1], 16); + } + + public ZooFatesStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID) + throws KeeperException, InterruptedException { + + this.path = path; + this.zk = zk; + this.defered = Collections.synchronizedMap(new HashMap<>()); + this.lockID = lockID; + + zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); + } + + /** + * For testing only + */ + ZooFatesStore() { + lockID = null; + } + + @Override + public long create() { + while (true) { + try { + // looking at the code for SecureRandom, it appears to be thread safe + long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL; + zk.putPersistentData(getTXPath(tid), new NodeValue(FateStatus.NEW, "", "").serialize(), + NodeExistsPolicy.FAIL); + return tid; + } catch (NodeExistsException nee) { + // exist, so just try another random # + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + } + + @Override + public FateStore reserve(long tid) { + var retry = + Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS) + .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry(); + + var resFateOp = tryReserve(tid); + while (resFateOp.isEmpty()) { + try { + retry.waitForNextAttempt(log, "Attempting to reserve " + FateTxId.formatTid(tid)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalArgumentException(e); + } + resFateOp = tryReserve(tid); + } + + retry.logCompletion(log, "Attempting to reserve " + FateTxId.formatTid(tid)); + return resFateOp.orElseThrow(); + } + + /** + * Attempt to reserve transaction + * + * @param tid transaction id + * @return true if reserved by this call, false if already reserved + */ + @Override + public Optional> tryReserve(long tid) { + + // uniquely identify this attempt to reserve the fate operation data + var uuid = UUID.randomUUID(); + + try { + byte[] newValue = zk.mutateExisting(getTXPath(tid), currentValue -> { + var nodeVal = new NodeValue(currentValue); + // The uuid handle 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 (!nodeVal.isReserved() || nodeVal.uuid.equals(uuid.toString())) { + return new NodeValue(nodeVal.status, lockID.serialize(""), uuid.toString()).serialize(); + } else { + // returning null will not change the value AND the null will be returned + return null; + } + }); + + if (newValue != null) { + // clear any deferment if it exists, it may or may not be set when its unreserved + defered.remove(tid); + return Optional.of(new FateStoreImpl(tid, uuid)); + } else { + return Optional.empty(); + } + } catch (NoNodeException e) { + log.trace("Fate op does not exists so can not reserve", e); + return Optional.empty(); + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } + } + + private class ReadOnlyFateStoreImpl implements ReadOnlyFateStore { + private static final int RETRIES = 10; + + protected final long tid; + + protected ReadOnlyFateStoreImpl(long tid) { + this.tid = tid; + } + + @Override + public Repo top() { + checkState(false); + for (int i = 0; i < RETRIES; i++) { + String txpath = getTXPath(tid); + try { + String top; + try { + top = findTop(txpath); + if (top == null) { + return null; + } + } catch (KeeperException.NoNodeException ex) { + throw new IllegalStateException(ex); + } + + byte[] ser = zk.getData(txpath + "/" + top); + @SuppressWarnings("unchecked") + var deserialized = (Repo) deserialize(ser); + return deserialized; + } catch (KeeperException.NoNodeException ex) { + log.debug("zookeeper error reading " + txpath + ": " + ex, ex); + sleepUninterruptibly(100, MILLISECONDS); + continue; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + return null; + } + + String findTop(String txpath) throws KeeperException, InterruptedException { + checkState(false); + List ops = zk.getChildren(txpath); + + ops = new ArrayList<>(ops); + + String max = ""; + + for (String child : ops) { + if (child.startsWith("repo_") && child.compareTo(max) > 0) { + max = child; + } + } + + if (max.equals("")) { + return null; + } + + return max; + } + + private FateStatus _getStatus() { + checkState(false); + try { + return new NodeValue(zk.getData(getTXPath(tid))).status; + } catch (NoNodeException nne) { + return FateStatus.UNKNOWN; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public FateStatus getStatus() { + checkState(false); + return _getStatus(); + } + + @Override + public FateStatus waitForStatusChange(EnumSet expected) { + checkState(false); + // 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... + var retry = Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS) + .incrementBy(25, MILLISECONDS).maxWait(1, SECONDS).backOffFactor(1.5) + .logInterval(3, MINUTES).createRetry(); + + while (true) { + FateStatus status = _getStatus(); + if (expected.contains(status)) { + retry.logCompletion(log, "Waiting on status change for " + FateTxId.formatTid(tid) + + " expected:" + expected + " status:" + status); + return status; + } + + try { + retry.waitForNextAttempt(log, "Waiting on status change for " + FateTxId.formatTid(tid) + + " expected:" + expected + " status:" + status); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + } + + @Override + public Serializable getTransactionInfo(Fate.TxInfo txInfo) { + checkState(false); + try { + byte[] data = zk.getData(getTXPath(tid) + "/" + txInfo); + + if (data[0] == 'O') { + byte[] sera = new byte[data.length - 2]; + System.arraycopy(data, 2, sera, 0, sera.length); + return (Serializable) deserialize(sera); + } else if (data[0] == 'S') { + return new String(data, 2, data.length - 2, UTF_8); + } else { + throw new IllegalStateException("Bad node data " + txInfo); + } + } catch (NoNodeException nne) { + return null; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public long timeCreated() { + checkState(false); + try { + Stat stat = zk.getZooKeeper().exists(getTXPath(tid), false); + return stat.getCtime(); + } catch (Exception e) { + return 0; + } + } + + @Override + public long getID() { + checkState(false); + return tid; + } + + @Override + public List> getStack() { + checkState(false); + String txpath = getTXPath(tid); + + outer: while (true) { + List ops; + try { + ops = zk.getChildren(txpath); + } catch (KeeperException.NoNodeException e) { + return Collections.emptyList(); + } catch (KeeperException | InterruptedException e1) { + throw new IllegalStateException(e1); + } + + ops = new ArrayList<>(ops); + ops.sort(Collections.reverseOrder()); + + ArrayList> dops = new ArrayList<>(); + + for (String child : ops) { + if (child.startsWith("repo_")) { + byte[] ser; + try { + ser = zk.getData(txpath + "/" + child); + @SuppressWarnings("unchecked") + var repo = (ReadOnlyRepo) deserialize(ser); + dops.add(repo); + } catch (KeeperException.NoNodeException e) { + // children changed so start over + continue outer; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + } + + return dops; + } + } + + protected void checkState(boolean unreserving) {} + + } + + private class FateStoreImpl extends ReadOnlyFateStoreImpl implements FateStore { + + private boolean reserved = true; + private boolean deleted = false; + + private final UUID uuid; + + protected FateStoreImpl(long tid, UUID uuid) { + super(tid); + this.uuid = Objects.requireNonNull(uuid); + } + + @Override + public void push(Repo repo) throws StackOverflowException { + checkState(false); + String txpath = getTXPath(tid); + try { + String top = findTop(txpath); + if (top != null && Long.parseLong(top.split("_")[1]) > 100) { + throw new StackOverflowException("Repo stack size too large"); + } + + zk.putPersistentSequential(txpath + "/repo_", serialize(repo)); + } catch (StackOverflowException soe) { + throw soe; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void pop() { + checkState(false); + try { + String txpath = getTXPath(tid); + String top = findTop(txpath); + if (top == null) { + throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid)); + } + zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void setStatus(FateStatus status) { + checkState(false); + try { + zk.mutateExisting(getTXPath(tid), currentValue -> { + var nodeVal = new NodeValue(currentValue); + Preconditions.checkState(nodeVal.uuid.equals(uuid.toString()), + "Tried to set status for %s and it was not reserved", FateTxId.formatTid(tid)); + return new NodeValue(status, nodeVal.lock, nodeVal.uuid).serialize(); + }); + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void delete() { + checkState(false); + try { + zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP); + deleted = true; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void setTransactionInfo(Fate.TxInfo txInfo, Serializable so) { + checkState(false); + try { + if (so instanceof String) { + zk.putPersistentData(getTXPath(tid) + "/" + txInfo, ("S " + so).getBytes(UTF_8), + NodeExistsPolicy.OVERWRITE); + } else { + byte[] sera = serialize(so); + byte[] data = new byte[sera.length + 2]; + System.arraycopy(sera, 0, data, 2, sera.length); + data[0] = 'O'; + data[1] = ' '; + zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE); + } + } catch (KeeperException | InterruptedException e2) { + throw new IllegalStateException(e2); + } + } + + private void unreserve() { + checkState(true); + try { + if (!deleted) { + zk.mutateExisting(getTXPath(tid), currentValue -> { + var nodeVal = new NodeValue(currentValue); + if (nodeVal.uuid.equals(uuid.toString())) { + return new NodeValue(nodeVal.status, "", "").serialize(); + } else { + // possible this is running a 2nd time in zk server fault conditions and its first + // write went through + return null; + } + }); + } + reserved = false; + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void unreserve(long deferTime) { + + if (deferTime < 0) { + throw new IllegalArgumentException("deferTime < 0 : " + deferTime); + } + + if (deferTime > 0) { + // add to defered before actually unreserving + defered.put(tid, System.currentTimeMillis() + deferTime); + } + + unreserve(); + } + + @Override + protected void checkState(boolean unreserving) { + super.checkState(unreserving); + if (!reserved) { + throw new IllegalStateException("Attempted to use fate store " + FateTxId.formatTid(getID()) + + " after unreserving it."); + } + + if (!unreserving && deleted) { + throw new IllegalStateException("Attempted to use fate store for " + + FateTxId.formatTid(getID()) + " after deleting it."); + } + } + } + + @Override + public ReadOnlyFateStore read(long tid) { + return new ReadOnlyFateStoreImpl(tid); + } + + @Override + public List list() { + try { + ArrayList l = new ArrayList<>(); + List transactions = zk.getChildren(path); + for (String txid : transactions) { + l.add(parseTid(txid)); + } + return l; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public Iterator runnable(PartitionData partitionData) { + try { + ArrayList runnableTids = new ArrayList<>(); + List transactions = zk.getChildren(path); + for (String txid : transactions) { + try { + var nodeVal = new NodeValue(zk.getData(path + "/" + txid)); + var tid = parseTid(txid); + if (!nodeVal.isReserved() + && (nodeVal.status == FateStatus.IN_PROGRESS + || nodeVal.status == FateStatus.FAILED_IN_PROGRESS + || nodeVal.status == FateStatus.SUBMITTED) + && tid % partitionData.getTotalInstances() == partitionData.getPartition()) { + runnableTids.add(tid); + } + } catch (NoNodeException nne) { + // expected race condition that node could be deleted after getting list of children + log.trace("Skipping missing node {}", txid); + } + } + + // Anything that is not runnable in this partition should be removed from the defered set + defered.keySet().retainAll(runnableTids); + + // Filter out any transactions that are not read to run because of deferment + runnableTids.removeIf(runnableTid -> { + var deferedTime = defered.get(runnableTid); + return deferedTime != null && deferedTime < System.currentTimeMillis(); + }); + + return runnableTids.iterator(); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java deleted file mode 100644 index 728cabcf7ee..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java +++ /dev/null @@ -1,574 +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; - -import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; -import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.apache.accumulo.core.util.LazySingletons.RANDOM; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.ArrayList; -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.Set; - -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; -import org.apache.accumulo.core.util.FastFormat; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.KeeperException.NodeExistsException; -import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - -//TODO use zoocache? - ACCUMULO-1297 -//TODO handle zookeeper being down gracefully - ACCUMULO-1297 - -public class ZooStore implements TStore { - - private static final Logger log = LoggerFactory.getLogger(ZooStore.class); - private String path; - private ZooReaderWriter zk; - private String lastReserved = ""; - private Set reserved; - private Map defered; - private long statusChangeEvents = 0; - private int reservationsWaiting = 0; - - private byte[] serialize(Object o) { - - try { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos); - oos.writeObject(o); - oos.close(); - - return baos.toByteArray(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @SuppressFBWarnings(value = "OBJECT_DESERIALIZATION", - justification = "unsafe to store arbitrary serialized objects like this, but needed for now" - + " for backwards compatibility") - private Object deserialize(byte[] ser) { - try { - ByteArrayInputStream bais = new ByteArrayInputStream(ser); - ObjectInputStream ois = new ObjectInputStream(bais); - return ois.readObject(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } catch (ReflectiveOperationException e) { - throw new IllegalStateException(e); - } - } - - private String getTXPath(long tid) { - return FastFormat.toHexString(path + "/tx_", tid, ""); - } - - private long parseTid(String txdir) { - return Long.parseLong(txdir.split("_")[1], 16); - } - - public ZooStore(String path, ZooReaderWriter zk) throws KeeperException, InterruptedException { - - this.path = path; - this.zk = zk; - this.reserved = new HashSet<>(); - this.defered = new HashMap<>(); - - zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); - } - - /** - * For testing only - */ - ZooStore() {} - - @Override - public long create() { - while (true) { - try { - // looking at the code for SecureRandom, it appears to be thread safe - long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL; - zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8), - NodeExistsPolicy.FAIL); - return tid; - } catch (NodeExistsException nee) { - // exist, so just try another random # - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - } - - @Override - public long reserve() { - try { - while (true) { - - long events; - synchronized (this) { - events = statusChangeEvents; - } - - List txdirs = new ArrayList<>(zk.getChildren(path)); - Collections.sort(txdirs); - - synchronized (this) { - if (!txdirs.isEmpty() && txdirs.get(txdirs.size() - 1).compareTo(lastReserved) <= 0) { - lastReserved = ""; - } - } - - for (String txdir : txdirs) { - long tid = parseTid(txdir); - - synchronized (this) { - // this check makes reserve pick up where it left off, so that it cycles through all as - // it is repeatedly called.... failing to do so can lead to - // starvation where fate ops that sort higher and hold a lock are never reserved. - if (txdir.compareTo(lastReserved) <= 0) { - continue; - } - - if (defered.containsKey(tid)) { - if (defered.get(tid) < System.currentTimeMillis()) { - defered.remove(tid); - } else { - continue; - } - } - if (reserved.contains(tid)) { - continue; - } else { - reserved.add(tid); - lastReserved = txdir; - } - } - - // have reserved id, status should not change - - try { - TStatus status = TStatus.valueOf(new String(zk.getData(path + "/" + txdir), UTF_8)); - if (status == TStatus.SUBMITTED || status == TStatus.IN_PROGRESS - || status == TStatus.FAILED_IN_PROGRESS) { - return tid; - } else { - unreserve(tid); - } - } catch (NoNodeException nne) { - // node deleted after we got the list of children, its ok - unreserve(tid); - } catch (KeeperException | InterruptedException | RuntimeException e) { - unreserve(tid); - throw e; - } - } - - synchronized (this) { - // suppress lgtm alert - synchronized variable is not always true - if (events == statusChangeEvents) { // lgtm [java/constant-comparison] - if (defered.isEmpty()) { - this.wait(5000); - } else { - Long minTime = Collections.min(defered.values()); - long waitTime = minTime - System.currentTimeMillis(); - if (waitTime > 0) { - this.wait(Math.min(waitTime, 5000)); - } - } - } - } - } - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void reserve(long tid) { - synchronized (this) { - reservationsWaiting++; - try { - while (reserved.contains(tid)) { - try { - this.wait(1000); - } catch (InterruptedException e) { - throw new IllegalStateException(e); - } - } - - reserved.add(tid); - } finally { - reservationsWaiting--; - } - } - } - - /** - * Attempt to reserve transaction - * - * @param tid transaction id - * @return true if reserved by this call, false if already reserved - */ - @Override - public boolean tryReserve(long tid) { - synchronized (this) { - if (!reserved.contains(tid)) { - reserve(tid); - return true; - } - return false; - } - } - - private void unreserve(long tid) { - synchronized (this) { - if (!reserved.remove(tid)) { - throw new IllegalStateException( - "Tried to unreserve id that was not reserved " + FateTxId.formatTid(tid)); - } - - // do not want this unreserve to unesc wake up threads in reserve()... this leads to infinite - // loop when tx is stuck in NEW... - // only do this when something external has called reserve(tid)... - if (reservationsWaiting > 0) { - this.notifyAll(); - } - } - } - - @Override - public void unreserve(long tid, long deferTime) { - - if (deferTime < 0) { - throw new IllegalArgumentException("deferTime < 0 : " + deferTime); - } - - synchronized (this) { - if (!reserved.remove(tid)) { - throw new IllegalStateException( - "Tried to unreserve id that was not reserved " + FateTxId.formatTid(tid)); - } - - if (deferTime > 0) { - defered.put(tid, System.currentTimeMillis() + deferTime); - } - - this.notifyAll(); - } - - } - - private void verifyReserved(long tid) { - synchronized (this) { - if (!reserved.contains(tid)) { - throw new IllegalStateException( - "Tried to operate on unreserved transaction " + FateTxId.formatTid(tid)); - } - } - } - - private static final int RETRIES = 10; - - @Override - public Repo top(long tid) { - verifyReserved(tid); - - for (int i = 0; i < RETRIES; i++) { - String txpath = getTXPath(tid); - try { - String top; - try { - top = findTop(txpath); - if (top == null) { - return null; - } - } catch (KeeperException.NoNodeException ex) { - throw new IllegalStateException(ex); - } - - byte[] ser = zk.getData(txpath + "/" + top); - @SuppressWarnings("unchecked") - var deserialized = (Repo) deserialize(ser); - return deserialized; - } catch (KeeperException.NoNodeException ex) { - log.debug("zookeeper error reading " + txpath + ": " + ex, ex); - sleepUninterruptibly(100, MILLISECONDS); - continue; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - return null; - } - - private String findTop(String txpath) throws KeeperException, InterruptedException { - List ops = zk.getChildren(txpath); - - ops = new ArrayList<>(ops); - - String max = ""; - - for (String child : ops) { - if (child.startsWith("repo_") && child.compareTo(max) > 0) { - max = child; - } - } - - if (max.equals("")) { - return null; - } - - return max; - } - - @Override - public void push(long tid, Repo repo) throws StackOverflowException { - verifyReserved(tid); - - String txpath = getTXPath(tid); - try { - String top = findTop(txpath); - if (top != null && Long.parseLong(top.split("_")[1]) > 100) { - throw new StackOverflowException("Repo stack size too large"); - } - - zk.putPersistentSequential(txpath + "/repo_", serialize(repo)); - } catch (StackOverflowException soe) { - throw soe; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void pop(long tid) { - verifyReserved(tid); - - try { - String txpath = getTXPath(tid); - String top = findTop(txpath); - if (top == null) { - throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid)); - } - zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP); - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - private TStatus _getStatus(long tid) { - try { - return TStatus.valueOf(new String(zk.getData(getTXPath(tid)), UTF_8)); - } catch (NoNodeException nne) { - return TStatus.UNKNOWN; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public TStatus getStatus(long tid) { - verifyReserved(tid); - return _getStatus(tid); - } - - @Override - public TStatus waitForStatusChange(long tid, EnumSet expected) { - while (true) { - long events; - synchronized (this) { - events = statusChangeEvents; - } - - TStatus status = _getStatus(tid); - if (expected.contains(status)) { - return status; - } - - synchronized (this) { - // suppress lgtm alert - synchronized variable is not always true - if (events == statusChangeEvents) { // lgtm [java/constant-comparison] - try { - this.wait(5000); - } catch (InterruptedException e) { - throw new IllegalStateException(e); - } - } - } - } - } - - @Override - public void setStatus(long tid, TStatus status) { - verifyReserved(tid); - - try { - zk.putPersistentData(getTXPath(tid), status.name().getBytes(UTF_8), - NodeExistsPolicy.OVERWRITE); - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - - synchronized (this) { - statusChangeEvents++; - } - - } - - @Override - public void delete(long tid) { - verifyReserved(tid); - - try { - zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP); - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void setTransactionInfo(long tid, Fate.TxInfo txInfo, Serializable so) { - verifyReserved(tid); - - try { - if (so instanceof String) { - zk.putPersistentData(getTXPath(tid) + "/" + txInfo, ("S " + so).getBytes(UTF_8), - NodeExistsPolicy.OVERWRITE); - } else { - byte[] sera = serialize(so); - byte[] data = new byte[sera.length + 2]; - System.arraycopy(sera, 0, data, 2, sera.length); - data[0] = 'O'; - data[1] = ' '; - zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE); - } - } catch (KeeperException | InterruptedException e2) { - throw new IllegalStateException(e2); - } - } - - @Override - public Serializable getTransactionInfo(long tid, Fate.TxInfo txInfo) { - verifyReserved(tid); - - try { - byte[] data = zk.getData(getTXPath(tid) + "/" + txInfo); - - if (data[0] == 'O') { - byte[] sera = new byte[data.length - 2]; - System.arraycopy(data, 2, sera, 0, sera.length); - return (Serializable) deserialize(sera); - } else if (data[0] == 'S') { - return new String(data, 2, data.length - 2, UTF_8); - } else { - throw new IllegalStateException("Bad node data " + txInfo); - } - } catch (NoNodeException nne) { - return null; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public List list() { - try { - ArrayList l = new ArrayList<>(); - List transactions = zk.getChildren(path); - for (String txid : transactions) { - l.add(parseTid(txid)); - } - return l; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public long timeCreated(long tid) { - verifyReserved(tid); - - try { - Stat stat = zk.getZooKeeper().exists(getTXPath(tid), false); - return stat.getCtime(); - } catch (Exception e) { - return 0; - } - } - - @Override - public List> getStack(long tid) { - String txpath = getTXPath(tid); - - outer: while (true) { - List ops; - try { - ops = zk.getChildren(txpath); - } catch (KeeperException.NoNodeException e) { - return Collections.emptyList(); - } catch (KeeperException | InterruptedException e1) { - throw new IllegalStateException(e1); - } - - ops = new ArrayList<>(ops); - ops.sort(Collections.reverseOrder()); - - ArrayList> dops = new ArrayList<>(); - - for (String child : ops) { - if (child.startsWith("repo_")) { - byte[] ser; - try { - ser = zk.getData(txpath + "/" + child); - @SuppressWarnings("unchecked") - var repo = (ReadOnlyRepo) deserialize(ser); - dops.add(repo); - } catch (KeeperException.NoNodeException e) { - // children changed so start over - continue outer; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - } - - return dops; - } - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java index 47d906fedb4..8eb4e17d62c 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java @@ -85,7 +85,6 @@ public LockID(String path, String node, long eid) { } public String serialize(String root) { - return path.substring(root.length()) + "/" + node + "$" + Long.toHexString(eid); } 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 fd31a95e6c8..01383f8155e 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 @@ -22,14 +22,18 @@ import java.io.Serializable; import java.util.EnumSet; +import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.function.Function; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; -import org.apache.accumulo.core.fate.TStore; +import org.apache.accumulo.core.manager.PartitionData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,49 +44,116 @@ public class FateLogger { // reproducible problems with FATE transactions. private static final Logger storeLog = LoggerFactory.getLogger(PREFIX + "store"); - public static TStore wrap(TStore store, Function,String> toLogString) { - - // only logging operations that change the persisted data, not operations that only read data - return new TStore<>() { - - @Override - public long reserve() { - return store.reserve(); - } - - @Override - public void reserve(long tid) { - store.reserve(tid); - } - - @Override - public boolean tryReserve(long tid) { - return store.tryReserve(tid); - } - - @Override - public void unreserve(long tid, long deferTime) { - store.unreserve(tid, deferTime); - } + private static class LoggingFateStore implements FatesStore.FateStore { + + private final FatesStore.FateStore wrapped; + private final Function,String> toLogString; + + private LoggingFateStore(FatesStore.FateStore wrapped, + Function,String> toLogString) { + this.wrapped = wrapped; + this.toLogString = toLogString; + } + + @Override + public List> getStack() { + return wrapped.getStack(); + } + + @Override + public ReadOnlyFatesStore.FateStatus getStatus() { + return wrapped.getStatus(); + } + + @Override + public ReadOnlyFatesStore.FateStatus + waitForStatusChange(EnumSet expected) { + return wrapped.waitForStatusChange(expected); + } + + @Override + public Serializable getTransactionInfo(Fate.TxInfo txInfo) { + return wrapped.getTransactionInfo(txInfo); + } + + @Override + public long timeCreated() { + return wrapped.timeCreated(); + } + + @Override + public long getID() { + return wrapped.getID(); + } + + @Override + public Repo top() { + return wrapped.top(); + } + + @Override + public void push(Repo repo) throws StackOverflowException { + wrapped.push(repo); + if (storeLog.isTraceEnabled()) { + storeLog.trace("{} pushed {}", formatTid(wrapped.getID()), toLogString.apply(repo)); + } + } + + @Override + public void pop() { + wrapped.pop(); + if (storeLog.isTraceEnabled()) { + storeLog.trace("{} popped", formatTid(wrapped.getID())); + } + } + + @Override + public void setStatus(ReadOnlyFatesStore.FateStatus status) { + wrapped.setStatus(status); + if (storeLog.isTraceEnabled()) { + storeLog.trace("{} setStatus to {}", formatTid(wrapped.getID()), status); + } + } + + @Override + public void setTransactionInfo(Fate.TxInfo txInfo, Serializable val) { + wrapped.setTransactionInfo(txInfo, val); + if (storeLog.isTraceEnabled()) { + storeLog.trace("{} setting {} to {}", formatTid(wrapped.getID()), txInfo, val); + } + } + + @Override + public void delete() { + wrapped.delete(); + if (storeLog.isTraceEnabled()) { + storeLog.trace("{} deleted fate transaction", formatTid(wrapped.getID())); + } + } + + @Override + public void unreserve(long deferTime) { + wrapped.unreserve(deferTime); + } + } - @Override - public List> getStack(long tid) { - return store.getStack(tid); - } + public static FatesStore wrap(FatesStore store, Function,String> toLogString) { + // only logging operations that change the persisted data, not operations that only read data + return new FatesStore<>() { @Override - public TStatus getStatus(long tid) { - return store.getStatus(tid); + public FateStore reserve(long tid) { + return new LoggingFateStore<>(store.reserve(tid), toLogString); } @Override - public TStatus waitForStatusChange(long tid, EnumSet expected) { - return store.waitForStatusChange(tid, expected); + public Optional> tryReserve(long tid) { + return store.tryReserve(tid).map(fos -> new LoggingFateStore<>(fos, toLogString)); } @Override - public Serializable getTransactionInfo(long tid, Fate.TxInfo txInfo) { - return store.getTransactionInfo(tid, txInfo); + public ReadOnlyFateStore read(long tid) { + return store.read(tid); } @Override @@ -91,8 +162,8 @@ public List list() { } @Override - public long timeCreated(long tid) { - return store.timeCreated(tid); + public Iterator runnable(PartitionData partitionData) { + return store.runnable(partitionData); } @Override @@ -103,51 +174,6 @@ public long create() { } return tid; } - - @Override - public Repo top(long tid) { - return store.top(tid); - } - - @Override - public void push(long tid, Repo repo) throws StackOverflowException { - store.push(tid, repo); - if (storeLog.isTraceEnabled()) { - storeLog.trace("{} pushed {}", formatTid(tid), toLogString.apply(repo)); - } - } - - @Override - public void pop(long tid) { - store.pop(tid); - if (storeLog.isTraceEnabled()) { - storeLog.trace("{} popped", formatTid(tid)); - } - } - - @Override - public void setStatus(long tid, TStatus status) { - store.setStatus(tid, status); - if (storeLog.isTraceEnabled()) { - storeLog.trace("{} setStatus to {}", formatTid(tid), status); - } - } - - @Override - public void setTransactionInfo(long tid, Fate.TxInfo txInfo, Serializable val) { - store.setTransactionInfo(tid, txInfo, val); - if (storeLog.isTraceEnabled()) { - storeLog.trace("{} setting {} to {}", formatTid(tid), txInfo, val); - } - } - - @Override - public void delete(long tid) { - store.delete(tid); - if (storeLog.isTraceEnabled()) { - storeLog.trace("{} deleted fate transaction", formatTid(tid)); - } - } }; } } diff --git a/core/src/main/java/org/apache/accumulo/core/manager/PartitionData.java b/core/src/main/java/org/apache/accumulo/core/manager/PartitionData.java new file mode 100644 index 00000000000..2f9fcfe7ce9 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/manager/PartitionData.java @@ -0,0 +1,92 @@ +/* + * 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.manager; + +import java.util.Objects; + +import com.google.common.base.Preconditions; + +/** + * This class is used by functional services inside the manager process to determine what action + * they should take when there are multiple active manager processes. + */ +public class PartitionData { + + /** + * Functional services within the manager that should only run in a single manager process. + */ + public enum SingletonManagerService { + FATE_LOCK_CLEANUP + } + + private final int partition; + private final int totalInstances; + + public PartitionData(int partition, int totalInstances) { + Preconditions.checkArgument(partition >= 0 && partition < totalInstances, + "Incorrect partition data, partition:%s totalInstances:%s", partition, totalInstances); + this.partition = partition; + this.totalInstances = totalInstances; + } + + /** + * Each manager instance in a cluster is assigned a unique partition number in the range + * [0,totalInstances). This method returns that unique partition number. + */ + public int getPartition() { + return partition; + } + + /** + * @return the total number of manager instances in a cluster. + */ + public int getTotalInstances() { + return totalInstances; + } + + @Override + public boolean equals(Object o) { + if (o instanceof PartitionData) { + var opd = (PartitionData) o; + return partition == opd.partition && totalInstances == opd.totalInstances; + } + + return false; + } + + @Override + public int hashCode() { + return Objects.hash(partition, totalInstances); + } + + @Override + public String toString() { + return partition + " " + totalInstances; + } + + /** + * Determines if a singleton manager service should run on this partition + */ + public boolean shouldRun(SingletonManagerService service) { + // The following should spread singleton manager services evenly over multiple manager + // processes, rather than running them all on partition zero. Its important that all + // partitions come to the same decision. + return (getPartition() + service.ordinal()) % getTotalInstances() == 0; + } +} diff --git a/core/src/test/java/org/apache/accumulo/core/fate/AgeOffStoreTest.java b/core/src/test/java/org/apache/accumulo/core/fate/AgeOffStoreTest.java deleted file mode 100644 index f36d7494b43..00000000000 --- a/core/src/test/java/org/apache/accumulo/core/fate/AgeOffStoreTest.java +++ /dev/null @@ -1,163 +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; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.util.HashSet; -import java.util.Set; - -import org.apache.accumulo.core.fate.AgeOffStore.TimeSource; -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; -import org.apache.zookeeper.KeeperException; -import org.junit.jupiter.api.Test; - -public class AgeOffStoreTest { - - private static class TestTimeSource implements TimeSource { - long time = 0; - - @Override - public long currentTimeMillis() { - return time; - } - - } - - @Test - public void testBasic() throws InterruptedException, KeeperException { - - TestTimeSource tts = new TestTimeSource(); - TestStore testStore = new TestStore(); - AgeOffStore aoStore = new AgeOffStore<>(testStore, 10, tts); - - aoStore.ageOff(); - - long txid1 = aoStore.create(); - aoStore.reserve(txid1); - aoStore.setStatus(txid1, TStatus.IN_PROGRESS); - aoStore.unreserve(txid1, 0); - - aoStore.ageOff(); - - long txid2 = aoStore.create(); - aoStore.reserve(txid2); - aoStore.setStatus(txid2, TStatus.IN_PROGRESS); - aoStore.setStatus(txid2, TStatus.FAILED); - aoStore.unreserve(txid2, 0); - - tts.time = 6; - - long txid3 = aoStore.create(); - aoStore.reserve(txid3); - aoStore.setStatus(txid3, TStatus.IN_PROGRESS); - aoStore.setStatus(txid3, TStatus.SUCCESSFUL); - aoStore.unreserve(txid3, 0); - - Long txid4 = aoStore.create(); - - aoStore.ageOff(); - - assertEquals(Set.of(txid1, txid2, txid3, txid4), new HashSet<>(aoStore.list())); - assertEquals(4, new HashSet<>(aoStore.list()).size()); - - tts.time = 15; - - aoStore.ageOff(); - - assertEquals(Set.of(txid1, txid3, txid4), new HashSet<>(aoStore.list())); - assertEquals(3, new HashSet<>(aoStore.list()).size()); - - tts.time = 30; - - aoStore.ageOff(); - - assertEquals(Set.of(txid1), new HashSet<>(aoStore.list())); - assertEquals(1, Set.of(aoStore.list()).size()); - } - - @Test - public void testNonEmpty() throws InterruptedException, KeeperException { - // test age off when source store starts off non empty - - TestTimeSource tts = new TestTimeSource(); - TestStore testStore = new TestStore(); - long txid1 = testStore.create(); - testStore.reserve(txid1); - testStore.setStatus(txid1, TStatus.IN_PROGRESS); - testStore.unreserve(txid1, 0); - - long txid2 = testStore.create(); - testStore.reserve(txid2); - testStore.setStatus(txid2, TStatus.IN_PROGRESS); - testStore.setStatus(txid2, TStatus.FAILED); - testStore.unreserve(txid2, 0); - - long txid3 = testStore.create(); - testStore.reserve(txid3); - testStore.setStatus(txid3, TStatus.IN_PROGRESS); - testStore.setStatus(txid3, TStatus.SUCCESSFUL); - testStore.unreserve(txid3, 0); - - Long txid4 = testStore.create(); - - AgeOffStore aoStore = new AgeOffStore<>(testStore, 10, tts); - - assertEquals(Set.of(txid1, txid2, txid3, txid4), new HashSet<>(aoStore.list())); - assertEquals(4, new HashSet<>(aoStore.list()).size()); - - aoStore.ageOff(); - - assertEquals(Set.of(txid1, txid2, txid3, txid4), new HashSet<>(aoStore.list())); - assertEquals(4, new HashSet<>(aoStore.list()).size()); - - tts.time = 15; - - aoStore.ageOff(); - - assertEquals(Set.of(txid1), new HashSet<>(aoStore.list())); - assertEquals(1, new HashSet<>(aoStore.list()).size()); - - aoStore.reserve(txid1); - aoStore.setStatus(txid1, TStatus.FAILED_IN_PROGRESS); - aoStore.unreserve(txid1, 0); - - tts.time = 30; - - aoStore.ageOff(); - - assertEquals(Set.of(txid1), new HashSet<>(aoStore.list())); - assertEquals(1, new HashSet<>(aoStore.list()).size()); - - aoStore.reserve(txid1); - aoStore.setStatus(txid1, TStatus.FAILED); - aoStore.unreserve(txid1, 0); - - aoStore.ageOff(); - - assertEquals(Set.of(txid1), new HashSet<>(aoStore.list())); - assertEquals(1, new HashSet<>(aoStore.list()).size()); - - tts.time = 42; - - aoStore.ageOff(); - - assertEquals(0, new HashSet<>(aoStore.list()).size()); - } -} 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 deleted file mode 100644 index 9f6d44b27ca..00000000000 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ /dev/null @@ -1,107 +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; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * Transient in memory store for transactions. - */ -public class TestStore extends ZooStore { - - private long nextId = 1; - private Map statuses = new HashMap<>(); - private Set reserved = new HashSet<>(); - - @Override - public long create() { - statuses.put(nextId, TStatus.NEW); - return nextId++; - } - - @Override - public void reserve(long tid) { - if (reserved.contains(tid)) { - throw new IllegalStateException(); // zoo store would wait, but do not expect test to reserve - } - // twice... if test change, then change this - reserved.add(tid); - } - - @Override - public boolean tryReserve(long tid) { - synchronized (this) { - if (!reserved.contains(tid)) { - reserve(tid); - return true; - } - return false; - } - } - - @Override - public void unreserve(long tid, long deferTime) { - if (!reserved.remove(tid)) { - throw new IllegalStateException(); - } - } - - @Override - public org.apache.accumulo.core.fate.TStore.TStatus getStatus(long tid) { - if (!reserved.contains(tid)) { - throw new IllegalStateException(); - } - - TStatus status = statuses.get(tid); - if (status == null) { - return TStatus.UNKNOWN; - } - return status; - } - - @Override - public void setStatus(long tid, org.apache.accumulo.core.fate.TStore.TStatus status) { - if (!reserved.contains(tid)) { - throw new IllegalStateException(); - } - if (!statuses.containsKey(tid)) { - throw new IllegalStateException(); - } - statuses.put(tid, status); - } - - @Override - public void delete(long tid) { - if (!reserved.contains(tid)) { - throw new IllegalStateException(); - } - statuses.remove(tid); - } - - @Override - public List list() { - return new ArrayList<>(statuses.keySet()); - } - -} 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 8dc2981a03c..9e1eda3f3aa 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 @@ -55,8 +55,8 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.FateTxId; -import org.apache.accumulo.core.fate.ReadOnlyTStore; -import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; @@ -761,7 +761,7 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); - ZooStore zs = new ZooStore<>(fateZkPath, zk); + ZooFatesStore zs = new ZooFatesStore<>(fateZkPath, zk, null); // TODO if (fateOpsCommand.cancel) { cancelSubmittedFateTxs(context, fateOpsCommand.txList); @@ -783,7 +783,8 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps if (fateOpsCommand.print) { final Set sortedTxs = new TreeSet<>(); fateOpsCommand.txList.forEach(s -> sortedTxs.add(parseTidFromUserInput(s))); - EnumSet statusFilter = getCmdLineStatusFilters(fateOpsCommand.states); + EnumSet statusFilter = + getCmdLineStatusFilters(fateOpsCommand.states); admin.print(zs, zk, zTableLocksPath, new Formatter(System.out), sortedTxs, statusFilter); // print line break at the end System.out.println(); @@ -835,7 +836,7 @@ private boolean cancelFateOperation(ClientContext context, long txid) throws Acc } private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUtil admin, - ReadOnlyTStore zs, ServiceLock.ServiceLockPath tableLocksPath) + ReadOnlyFatesStore zs, ServiceLock.ServiceLockPath tableLocksPath) throws InterruptedException, AccumuloException, AccumuloSecurityException, KeeperException { ZooReaderWriter zk = context.getZooReaderWriter(); @@ -854,7 +855,7 @@ private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUti } }); - EnumSet statusFilter = getCmdLineStatusFilters(cmd.states); + EnumSet statusFilter = getCmdLineStatusFilters(cmd.states); FateSummaryReport report = new FateSummaryReport(idsToNameMap, statusFilter); @@ -881,12 +882,12 @@ private void printLines(List lines) { * * @return a set of status filters, or an empty set if none provides */ - private EnumSet getCmdLineStatusFilters(List states) { - EnumSet statusFilter = null; + private EnumSet getCmdLineStatusFilters(List states) { + EnumSet statusFilter = null; if (!states.isEmpty()) { - statusFilter = EnumSet.noneOf(ReadOnlyTStore.TStatus.class); + statusFilter = EnumSet.noneOf(ReadOnlyFatesStore.FateStatus.class); for (String element : states) { - statusFilter.add(ReadOnlyTStore.TStatus.valueOf(element)); + statusFilter.add(ReadOnlyFatesStore.FateStatus.valueOf(element)); } } return statusFilter; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java index d74c5ac6694..f6550b31882 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java @@ -33,7 +33,7 @@ import java.util.TreeSet; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyTStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -55,7 +55,7 @@ public class FateSummaryReport { private final transient Map idsToNameMap; public FateSummaryReport(Map idsToNameMap, - EnumSet statusFilter) { + EnumSet statusFilter) { this.idsToNameMap = idsToNameMap; if (statusFilter != null) { statusFilter.forEach(f -> this.statusFilterNames.add(f.name())); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java index 40cc9553058..5ab3f6098c7 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyTStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,7 +67,7 @@ public void noTablenameReport() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyTStore.TStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn(null).anyTimes(); expect(status1.getTxName()).andReturn(null).anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java index f171a7e3e8d..8aa27783e86 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java @@ -33,7 +33,7 @@ import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyTStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +50,7 @@ void orderingByDuration() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyTStore.TStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn("step1").anyTimes(); expect(status1.getTxName()).andReturn("runningTx1").anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); @@ -59,7 +59,7 @@ void orderingByDuration() { AdminUtil.TransactionStatus status2 = createMock(AdminUtil.TransactionStatus.class); expect(status2.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(7)).anyTimes(); - expect(status2.getStatus()).andReturn(ReadOnlyTStore.TStatus.IN_PROGRESS).anyTimes(); + expect(status2.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status2.getTop()).andReturn("step2").anyTimes(); expect(status2.getTxName()).andReturn("runningTx2").anyTimes(); expect(status2.getTxid()).andReturn("123456789").anyTimes(); @@ -93,7 +93,7 @@ public void lockTest() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyTStore.TStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn("step1").anyTimes(); expect(status1.getTxName()).andReturn("runningTx").anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index e94e4907bc9..1a5debb27ba 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -68,7 +68,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TRange; -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus; import org.apache.accumulo.core.manager.thrift.BulkImportState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; @@ -823,8 +823,8 @@ public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long o throws ThriftSecurityException, ThriftTableOperationException { authenticate(credentials); - TStatus status = manager.fate().waitForCompletion(opid); - if (status == TStatus.FAILED) { + FateStatus status = manager.fate().waitForCompletion(opid); + if (status == FateStatus.FAILED) { Exception e = manager.fate().getException(opid); if (e instanceof ThriftTableOperationException) { throw (ThriftTableOperationException) e; 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 6608b566381..7da6a92537c 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 @@ -22,7 +22,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySortedMap; -import static java.util.concurrent.TimeUnit.HOURS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -71,8 +70,8 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.dataImpl.KeyExtent; -import org.apache.accumulo.core.fate.AgeOffStore; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.lock.ServiceLock; @@ -82,6 +81,7 @@ import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; +import org.apache.accumulo.core.manager.PartitionData; import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl; import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl; import org.apache.accumulo.core.manager.balancer.TServerStatusImpl; @@ -255,6 +255,12 @@ public boolean stillManager() { return getManagerState() != ManagerState.STOP; } + // ELASTICITY_TODO implement this and determine behavior when manager does not have lock. Maybe it + // should return Optional, so it can return Optional.empty() when lock is not held? + public PartitionData getParitionData() { + return new PartitionData(0, 1); + } + /** * Retrieve the Fate object, blocking until it is ready. This could cause problems if Fate * operations are attempted to be used prior to the Manager being ready for them. If these @@ -1076,17 +1082,13 @@ boolean canSuspendTablets() { } try { - final AgeOffStore store = new AgeOffStore<>( - new org.apache.accumulo.core.fate.ZooStore<>(getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter()), - HOURS.toMillis(8), System::currentTimeMillis); + final ZooFatesStore store = new ZooFatesStore<>(getZooKeeperRoot() + Constants.ZFATE, + context.getZooReaderWriter(), managerLock.getLockID()); - Fate f = new Fate<>(this, store, TraceRepo::toLogString, getConfiguration()); + Fate f = new Fate<>(this, store, TraceRepo::toLogString, this::getParitionData, + getConfiguration()); fateRef.set(f); fateReadyLatch.countDown(); - - ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() - .scheduleWithFixedDelay(store::ageOff, 63000, 63000, MILLISECONDS)); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Exception setting up FaTE cleanup thread", e); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java index 57a561aa7b4..08372aa9e63 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java @@ -24,7 +24,7 @@ import java.util.TreeMap; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyTStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -101,7 +101,7 @@ Map getOpTypeCounters() { * @return the current FATE metric values. */ public static FateMetricValues getFromZooKeeper(final ServerContext context, - final String fateRootPath, final ReadOnlyTStore zooStore) { + final String fateRootPath, final ReadOnlyFatesStore zooStore) { FateMetricValues.Builder builder = FateMetricValues.builder(); @@ -116,7 +116,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, // states are enumerated - create new map with counts initialized to 0. Map states = new TreeMap<>(); - for (ReadOnlyTStore.TStatus t : ReadOnlyTStore.TStatus.values()) { + for (ReadOnlyFatesStore.FateStatus t : ReadOnlyFatesStore.FateStatus.values()) { states.put(t.name(), 0L); } @@ -132,7 +132,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, states.merge(stateName, 1L, Long::sum); // incr count for op type for for in_progress transactions. - if (ReadOnlyTStore.TStatus.IN_PROGRESS.equals(tx.getStatus())) { + if (ReadOnlyFatesStore.FateStatus.IN_PROGRESS.equals(tx.getStatus())) { String opType = tx.getTxName(); if (opType == null || opType.isEmpty()) { opType = "UNKNOWN"; @@ -189,7 +189,7 @@ static class Builder { // states are enumerated - create new map with counts initialized to 0. txStateCounters = new TreeMap<>(); - for (ReadOnlyTStore.TStatus t : ReadOnlyTStore.TStatus.values()) { + for (ReadOnlyFatesStore.FateStatus t : ReadOnlyFatesStore.FateStatus.values()) { txStateCounters.put(t.name(), 0L); } 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 758884d5b46..7558cb68b01 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 @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.fate.ReadOnlyTStore; -import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.metrics.MetricsUtil; import org.apache.accumulo.core.util.threads.ThreadPools; @@ -49,7 +49,7 @@ public class FateMetrics implements MetricsProducer { private static final String OP_TYPE_TAG = "op.type"; private final ServerContext context; - private final ReadOnlyTStore zooStore; + private final ReadOnlyFatesStore zooStore; private final String fateRootPath; private final long refreshDelay; @@ -71,7 +71,7 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - this.zooStore = new ZooStore<>(fateRootPath, context.getZooReaderWriter()); + this.zooStore = new ZooFatesStore<>(fateRootPath, context.getZooReaderWriter(), null); // TODO } catch (KeeperException ex) { throw new IllegalStateException( "FATE Metrics - Failed to create zoo store - metrics unavailable", ex); @@ -93,7 +93,7 @@ private void update() { fateErrorsGauge.set(metricValues.getZkConnectionErrors()); for (Entry vals : metricValues.getTxStateCounters().entrySet()) { - switch (ReadOnlyTStore.TStatus.valueOf(vals.getKey())) { + switch (ReadOnlyFatesStore.FateStatus.valueOf(vals.getKey())) { case NEW: newTxGauge.set(vals.getValue()); break; @@ -134,20 +134,22 @@ public void registerMetrics(final MeterRegistry registry) { fateErrorsGauge = registry.gauge(METRICS_FATE_ERRORS, Tags.concat(MetricsUtil.getCommonTags(), "type", "zk.connection"), new AtomicLong(0)); newTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyTStore.TStatus.NEW.name().toLowerCase()), new AtomicLong(0)); + ReadOnlyFatesStore.FateStatus.NEW.name().toLowerCase()), new AtomicLong(0)); submittedTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyTStore.TStatus.SUBMITTED.name().toLowerCase()), new AtomicLong(0)); - inProgressTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyTStore.TStatus.IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); - failedInProgressTxGauge = + "state", ReadOnlyFatesStore.FateStatus.SUBMITTED.name().toLowerCase()), new AtomicLong(0)); + inProgressTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyTStore.TStatus.FAILED_IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); + ReadOnlyFatesStore.FateStatus.IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); + failedInProgressTxGauge = registry.gauge(METRICS_FATE_TX, + Tags.concat(MetricsUtil.getCommonTags(), "state", + ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS.name().toLowerCase()), + new AtomicLong(0)); failedTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyTStore.TStatus.FAILED.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFatesStore.FateStatus.FAILED.name().toLowerCase()), new AtomicLong(0)); successfulTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyTStore.TStatus.SUCCESSFUL.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFatesStore.FateStatus.SUCCESSFUL.name().toLowerCase()), new AtomicLong(0)); unknownTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyTStore.TStatus.UNKNOWN.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFatesStore.FateStatus.UNKNOWN.name().toLowerCase()), new AtomicLong(0)); update(); 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 dedda83872b..f7699c96442 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 @@ -32,8 +32,8 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.fate.ReadOnlyTStore; -import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.volume.Volume; @@ -268,8 +268,24 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - final ReadOnlyTStore fate = new ZooStore<>( - context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()); + final ReadOnlyFatesStore fate = new ZooFatesStore<>( + context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter(), null); // TODO + // maybe + // have + // a + // version + // of + // zoostore + // w/o + // a + // lock + // that + // fails + // on + // ops + // that + // need + // lock if (!fate.list().isEmpty()) { throw new AccumuloException("Aborting upgrade because there are" + " outstanding FATE transactions from a previous Accumulo version." diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index 5834179fc09..74c7dc6db44 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -19,12 +19,12 @@ package org.apache.accumulo.test.fate.zookeeper; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED_IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.NEW; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.SUCCESSFUL; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.NEW; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUCCESSFUL; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -44,19 +44,21 @@ import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.fate.AgeOffStore; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateTxId; -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus; import org.apache.accumulo.core.fate.Repo; -import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.manager.PartitionData; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.ManagerRepo; import org.apache.accumulo.manager.tableOps.TraceRepo; import org.apache.accumulo.manager.tableOps.Utils; import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.test.util.Wait; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; import org.apache.zookeeper.KeeperException; import org.junit.jupiter.api.AfterAll; @@ -143,13 +145,16 @@ public static void teardown() throws Exception { szk.close(); } + private ZooUtil.LockID createLockID() { + return new ZooUtil.LockID("S1", "N1", 1234); + } + @Test @Timeout(30) public void testTransactionStatus() throws Exception { - final ZooStore zooStore = new ZooStore(ZK_ROOT + Constants.ZFATE, zk); - final AgeOffStore store = - new AgeOffStore(zooStore, 3000, System::currentTimeMillis); + final ZooFatesStore store = + new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -161,7 +166,8 @@ public void testTransactionStatus() throws Exception { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - Fate fate = new Fate(manager, store, TraceRepo::toLogString, config); + Fate fate = new Fate(manager, store, TraceRepo::toLogString, + () -> new PartitionData(0, 1), config); try { // Wait for the transaction runner to be scheduled. @@ -171,16 +177,16 @@ public void testTransactionStatus() throws Exception { finishCall = new CountDownLatch(1); long txid = fate.startTransaction(); - assertEquals(TStatus.NEW, getTxStatus(zk, txid)); + assertEquals(FateStatus.NEW, getTxStatus(zk, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); - assertEquals(TStatus.SUBMITTED, getTxStatus(zk, txid)); + assertEquals(FateStatus.SUBMITTED, getTxStatus(zk, txid)); // wait for call() to be called callStarted.await(); assertEquals(IN_PROGRESS, getTxStatus(zk, txid)); // tell the op to exit the method finishCall.countDown(); // Check that it transitions to SUCCESSFUL - TStatus s = getTxStatus(zk, txid); + FateStatus s = getTxStatus(zk, txid); while (s != SUCCESSFUL) { s = getTxStatus(zk, txid); Thread.sleep(10); @@ -207,9 +213,8 @@ public void testTransactionStatus() throws Exception { @Test public void testCancelWhileNew() throws Exception { - final ZooStore zooStore = new ZooStore(ZK_ROOT + Constants.ZFATE, zk); - final AgeOffStore store = - new AgeOffStore(zooStore, 3000, System::currentTimeMillis); + final ZooFatesStore store = + new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -221,7 +226,8 @@ public void testCancelWhileNew() throws Exception { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - Fate fate = new Fate(manager, store, TraceRepo::toLogString, config); + Fate fate = new Fate(manager, store, TraceRepo::toLogString, + () -> new PartitionData(0, 1), config); try { // Wait for the transaction runner to be scheduled. @@ -238,6 +244,8 @@ public void testCancelWhileNew() throws Exception { assertTrue(FAILED_IN_PROGRESS == getTxStatus(zk, txid) || FAILED == getTxStatus(zk, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); assertTrue(FAILED_IN_PROGRESS == getTxStatus(zk, txid) || FAILED == getTxStatus(zk, txid)); + // can not delete a failed in progress operation, so wait for it to transition to FAILED + Wait.waitFor(() -> FAILED_IN_PROGRESS != getTxStatus(zk, txid)); fate.delete(txid); } finally { fate.shutdown(); @@ -246,9 +254,8 @@ public void testCancelWhileNew() throws Exception { @Test public void testCancelWhileSubmittedAndRunning() throws Exception { - final ZooStore zooStore = new ZooStore(ZK_ROOT + Constants.ZFATE, zk); - final AgeOffStore store = - new AgeOffStore(zooStore, 3000, System::currentTimeMillis); + final ZooFatesStore store = + new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -260,7 +267,8 @@ public void testCancelWhileSubmittedAndRunning() throws Exception { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - Fate fate = new Fate(manager, store, TraceRepo::toLogString, config); + Fate fate = new Fate(manager, store, TraceRepo::toLogString, + () -> new PartitionData(0, 1), config); try { // Wait for the transaction runner to be scheduled. @@ -274,12 +282,11 @@ public void testCancelWhileSubmittedAndRunning() throws Exception { assertEquals(NEW, getTxStatus(zk, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); assertEquals(SUBMITTED, getTxStatus(zk, txid)); + callStarted.await(); // This is false because the transaction runner has reserved the FaTe // transaction. assertFalse(fate.cancel(txid)); - callStarted.await(); finishCall.countDown(); - fate.delete(txid); } finally { fate.shutdown(); } @@ -287,9 +294,8 @@ public void testCancelWhileSubmittedAndRunning() throws Exception { @Test public void testCancelWhileInCall() throws Exception { - final ZooStore zooStore = new ZooStore(ZK_ROOT + Constants.ZFATE, zk); - final AgeOffStore store = - new AgeOffStore(zooStore, 3000, System::currentTimeMillis); + final ZooFatesStore store = + new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -301,7 +307,8 @@ public void testCancelWhileInCall() throws Exception { ConfigurationCopy config = new ConfigurationCopy(); config.set(Property.GENERAL_THREADPOOL_SIZE, "2"); config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1"); - Fate fate = new Fate(manager, store, TraceRepo::toLogString, config); + Fate fate = new Fate(manager, store, TraceRepo::toLogString, + () -> new PartitionData(0, 1), config); try { // Wait for the transaction runner to be scheduled. @@ -336,11 +343,11 @@ private static void inCall() throws InterruptedException { * Get the status of the TX from ZK directly. Unable to call ZooStore.getStatus because this test * thread does not have the reservation (the FaTE thread does) */ - private static TStatus getTxStatus(ZooReaderWriter zrw, long txid) + private static FateStatus getTxStatus(ZooReaderWriter zrw, long txid) throws KeeperException, InterruptedException { zrw.sync(ZK_ROOT); String txdir = String.format("%s%s/tx_%016x", ZK_ROOT, Constants.ZFATE, txid); - return TStatus.valueOf(new String(zrw.getData(txdir), UTF_8)); + return FateStatus.valueOf(new String(zrw.getData(txdir), UTF_8).split(":")[0]); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java new file mode 100644 index 00000000000..dafb59753fd --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java @@ -0,0 +1,246 @@ +/* + * 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.zookeeper; + +import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; +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 java.io.File; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.accumulo.core.conf.DefaultConfiguration; +import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.manager.PartitionData; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; + +import com.google.common.collect.Sets; + +@Tag(ZOOKEEPER_TESTING_SERVER) +public class MultipleFateInstancesIT { + + @TempDir + private static File tempDir; + private static ZooKeeperTestingServer szk = null; + private static ZooReaderWriter zk; + + private static final String FATE_DIR = "/fate"; + + @BeforeAll + public static void setup() throws Exception { + szk = new ZooKeeperTestingServer(tempDir); + zk = szk.getZooReaderWriter(); + } + + @AfterAll + public static void teardown() throws Exception { + szk.close(); + } + + @Test + @Timeout(30) + public void testZooStores() throws Exception { + ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + final ZooFatesStore zooFatesStore1 = new ZooFatesStore(FATE_DIR, zk, lock1); + final ZooFatesStore zooFatesStore2 = new ZooFatesStore(FATE_DIR, zk, lock2); + + Set allIds = new HashSet<>(); + + for (int i = 0; i < 100; i++) { + assertTrue(allIds.add(zooFatesStore1.create())); + assertTrue(allIds.add(zooFatesStore2.create())); + } + + var pd1 = new PartitionData(0, 2); + var pd2 = new PartitionData(1, 2); + + // nothing should be ready to run + assertFalse(zooFatesStore1.runnable(pd1).hasNext()); + assertFalse(zooFatesStore2.runnable(pd2).hasNext()); + + for (var txid : allIds) { + if (txid % 2 == 0) { + var rfo = zooFatesStore1.reserve(txid); + assertTrue(zooFatesStore2.tryReserve(txid).isEmpty()); + rfo.setStatus(ReadOnlyFatesStore.FateStatus.SUBMITTED); + rfo.unreserve(0); + } else { + var rfo = zooFatesStore2.reserve(txid); + assertTrue(zooFatesStore1.tryReserve(txid).isEmpty()); + rfo.setStatus(ReadOnlyFatesStore.FateStatus.SUBMITTED); + rfo.unreserve(0); + } + } + + HashSet runnable1 = new HashSet<>(); + zooFatesStore1.runnable(pd1).forEachRemaining(txid -> assertTrue(runnable1.add(txid))); + HashSet runnable2 = new HashSet<>(); + zooFatesStore2.runnable(pd2).forEachRemaining(txid -> assertTrue(runnable2.add(txid))); + + assertFalse(runnable1.isEmpty()); + assertFalse(runnable2.isEmpty()); + assertTrue(Collections.disjoint(runnable1, runnable2)); + assertEquals(allIds, Sets.union(runnable1, runnable2)); + + for (var txid : allIds) { + var rfo = zooFatesStore1.reserve(txid); + assertTrue(zooFatesStore2.tryReserve(txid).isEmpty()); + rfo.delete(); + rfo.unreserve(0); + } + } + + public static class TestEnv { + public final Set executedOps = Collections.synchronizedSet(new HashSet<>()); + + } + + public static class TestRepo implements Repo { + + private static final long serialVersionUID = 1L; + + @Override + public long isReady(long tid, TestEnv environment) throws Exception { + return 0; + } + + @Override + public String getName() { + return null; + } + + @Override + public Repo call(long tid, TestEnv environment) throws Exception { + environment.executedOps.add(tid); + return null; + } + + @Override + public void undo(long tid, TestEnv environment) throws Exception { + + } + + @Override + public String getReturn() { + return null; + } + } + + @Test + public void testMultipleFateInstances() throws Exception { + ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); + ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); + final ZooFatesStore zooFatesStore1 = new ZooFatesStore<>(FATE_DIR, zk, lock1); + final ZooFatesStore zooFatesStore2 = new ZooFatesStore<>(FATE_DIR, zk, lock2); + + TestEnv testEnv1 = new TestEnv(); + TestEnv testEnv2 = new TestEnv(); + + AtomicReference partitionData1 = new AtomicReference<>(new PartitionData(0, 2)); + AtomicReference partitionData2 = new AtomicReference<>(new PartitionData(1, 2)); + + Fate fate1 = new Fate<>(testEnv1, zooFatesStore1, r -> "", partitionData1::get, + DefaultConfiguration.getInstance()); + Fate fate2 = new Fate<>(testEnv2, zooFatesStore2, r -> "", partitionData2::get, + DefaultConfiguration.getInstance()); + + Set submittedIds = new HashSet<>(); + + // submit 100 operations through fate1 instance. Even though all were submitted through a single + // instance, both instances should pick them up and run. + for (int i = 0; i < 100; i++) { + var id = fate1.startTransaction(); + fate1.seedTransaction("op" + i, id, new TestRepo(), true, "test"); + submittedIds.add(id); + } + + assertEquals(100, submittedIds.size()); + + // should be able to wait for completion on any fate instance + for (var id : submittedIds) { + fate2.waitForCompletion(id); + } + + // verify all fate ops ran + assertEquals(submittedIds, Sets.union(testEnv1.executedOps, testEnv2.executedOps)); + // verify each op only ran in one fate instance + assertTrue(Collections.disjoint(testEnv1.executedOps, testEnv2.executedOps)); + // verify both fate instances executed operations + assertFalse(testEnv1.executedOps.isEmpty()); + assertFalse(testEnv2.executedOps.isEmpty()); + + // create a third fate instance + ZooUtil.LockID lock3 = new ZooUtil.LockID("/locks", "L3", 54); + final ZooFatesStore zooFatesStore3 = new ZooFatesStore<>(FATE_DIR, zk, lock3); + TestEnv testEnv3 = new TestEnv(); + AtomicReference partitionData3 = new AtomicReference<>(new PartitionData(2, 3)); + Fate fate3 = new Fate<>(testEnv3, zooFatesStore3, r -> "", partitionData3::get, + DefaultConfiguration.getInstance()); + + // adjust the partition data for the existing fate instances, they should react to this change + partitionData1.set(new PartitionData(0, 3)); + partitionData2.set(new PartitionData(1, 3)); + + // clear the tracking sets from the last batch of operations executed + submittedIds.clear(); + testEnv1.executedOps.clear(); + testEnv2.executedOps.clear(); + + // execute another set of 100 operations, all three fate instances should pick these ops up + for (int i = 0; i < 100; i++) { + var id = fate3.startTransaction(); + fate3.seedTransaction("op2." + i, id, new TestRepo(), true, "test"); + submittedIds.add(id); + } + + // should be able to wait for completion on any fate instance + for (var id : submittedIds) { + fate1.waitForCompletion(id); + } + + // verify all fate ops ran + assertEquals(submittedIds, + Sets.union(Sets.union(testEnv1.executedOps, testEnv2.executedOps), testEnv3.executedOps)); + // verify each op only ran in one fate instance + assertTrue(Collections.disjoint(testEnv1.executedOps, testEnv2.executedOps)); + assertTrue(Collections.disjoint(testEnv1.executedOps, testEnv3.executedOps)); + assertTrue(Collections.disjoint(testEnv2.executedOps, testEnv3.executedOps)); + // verify all fate instances executed operations + assertFalse(testEnv1.executedOps.isEmpty()); + assertFalse(testEnv2.executedOps.isEmpty()); + assertFalse(testEnv3.executedOps.isEmpty()); + } +} 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 2283132c3f7..984fa796547 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 @@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; @@ -254,7 +254,8 @@ public void getFateStatus() { InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - ZooStore zs = new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); + ZooFatesStore zs = + new ZooFatesStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); @@ -344,7 +345,8 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - ZooStore zs = new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); + ZooFatesStore zs = + new ZooFatesStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); AdminUtil.FateStatus fateStatus = admin.getStatus(zs, zk, lockPath, null, null); @@ -502,4 +504,8 @@ public void multipleCompactions() { }); } + + private ZooUtil.LockID createLockID() { + return new ZooUtil.LockID("S1", "N1", 1234); + } } 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 175533f0c93..ad606b87b11 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 @@ -58,7 +58,7 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.AdminUtil.FateStatus; -import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.metadata.MetadataTable; @@ -226,7 +226,8 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); ZooReaderWriter zk = context.getZooReaderWriter(); - ZooStore zs = new ZooStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk); + ZooFatesStore zs = + new ZooFatesStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, null); // TODO var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); return admin.getStatus(zs, zk, lockPath, null, null); } catch (KeeperException | InterruptedException e) { From 7196a43e9dd66cf5507c1685b8e0efb95ae4e961 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Mon, 20 Nov 2023 18:52:00 -0500 Subject: [PATCH 02/12] fix javadoc --- .../org/apache/accumulo/core/fate/ReadOnlyFatesStore.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java index 69f830811f4..d25e3c552ac 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java @@ -59,7 +59,6 @@ interface ReadOnlyFateStore { * * Caller must have already reserved tid. * - * @param tid transaction id, previously reserved. * @return a read-only view of the operation */ ReadOnlyRepo top(); @@ -75,7 +74,6 @@ interface ReadOnlyFateStore { * * Caller must have already reserved tid. * - * @param tid transaction id, previously reserved. * @return execution status */ FateStatus getStatus(); @@ -83,7 +81,6 @@ interface ReadOnlyFateStore { /** * Wait for the status of a transaction to change * - * @param tid transaction id, need not have been reserved. * @param expected a set of possible statuses we are interested in being notified about. may not * be null. * @return execution status. @@ -95,7 +92,6 @@ interface ReadOnlyFateStore { * * Caller must have already reserved tid. * - * @param tid transaction id, previously reserved. * @param txInfo name of attribute of a transaction to retrieve. */ Serializable getTransactionInfo(Fate.TxInfo txInfo); From 66ace153fb18818d12448b8d75904a80ad2c2ad5 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Mon, 20 Nov 2023 21:07:02 -0500 Subject: [PATCH 03/12] fix javadoc --- .../main/java/org/apache/accumulo/core/fate/FatesStore.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java index 20aa969de35..4d4a8fbf8ae 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java @@ -21,6 +21,8 @@ import java.io.Serializable; import java.util.Optional; +import org.apache.accumulo.core.manager.PartitionData; + /** * FatesStore : a place to store fate data for all fate operations. * @@ -85,8 +87,8 @@ interface FateStore extends ReadOnlyFateStore { * upon successful return the store now controls the referenced transaction id. caller should no * longer interact with it. * - * @param deferTime time in millis to keep this transaction out of the pool used in the - * {@link #reserve() reserve} method. must be non-negative. + * @param deferTime time in millis to keep this transaction from being returned by + * {@link #runnable(PartitionData)}. Must be non-negative. */ void unreserve(long deferTime); } From a0f94d39651c86f186d0f1ecbe1dba7d51dc140b Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Tue, 21 Nov 2023 20:29:06 -0500 Subject: [PATCH 04/12] fix bug --- .../org/apache/accumulo/core/fate/Fate.java | 29 +++++++++++++++---- 1 file changed, 24 insertions(+), 5 deletions(-) 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 fe018f7a33e..459d5cd1eb3 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 @@ -31,7 +31,10 @@ import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.UNKNOWN; import static org.apache.accumulo.core.util.ShutdownUtil.isIOException; +import java.util.Collections; import java.util.EnumSet; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; @@ -77,6 +80,7 @@ public class Fate { private final AtomicBoolean keepRunning = new AtomicBoolean(true); private final Supplier partitionDataSupplier; private final BlockingQueue workQueue; + private final Set queuedWork; private final Thread workFinder; public enum TxInfo { @@ -130,13 +134,23 @@ public void run() { } lastPartitionData = partitionData; + if (workQueue.isEmpty()) { + queuedWork.clear(); + } + var iter = store.runnable(partitionData); - int count = 0; + int added = 0; while (iter.hasNext() && keepRunning.get()) { Long txid = iter.next(); - count++; try { + if (queuedWork.contains(txid)) { + // Avoid adding the same id to the queue multiple times as this will cause + // unnecessary work when multiple threads try to work on the same id. + continue; + } + added++; + queuedWork.add(txid); while (keepRunning.get()) { if (workQueue.offer(txid, 100, TimeUnit.MILLISECONDS)) { break; @@ -148,8 +162,8 @@ public void run() { } } - if (count == 0) { - // When nothing was found in the store, then start doing exponential backoff before + if (added == 0) { + // When nothing was added to the queue, then start doing exponential backoff before // reading from the store again in order to avoid overwhelming the store. try { retry.waitForNextAttempt(log, @@ -170,6 +184,9 @@ public void run() { } else { log.debug("Failure while attempting to find work for fate", e); } + + queuedWork.clear(); + workQueue.clear(); } } } @@ -186,6 +203,7 @@ public void run() { if (unreservedTid == null) { continue; } + queuedWork.remove(unreservedTid); var optionalopStore = store.tryReserve(unreservedTid); if (optionalopStore.isPresent()) { opStore = optionalopStore.orElseThrow(); @@ -196,7 +214,7 @@ public void run() { Repo op = opStore.top(); if (status == FAILED_IN_PROGRESS) { processFailed(opStore, op); - } else { + } else if (status == SUBMITTED || status == IN_PROGRESS) { Repo prevOp = null; try { deferTime = op.isReady(opStore.getID(), environment); @@ -345,6 +363,7 @@ public Fate(T environment, FatesStore store, Function,String> toLogSt // TODO this queue does not resize when config changes this.workQueue = new ArrayBlockingQueue<>(conf.getCount(Property.MANAGER_FATE_THREADPOOL_SIZE) * 4); + this.queuedWork = Collections.synchronizedSet(new HashSet<>()); this.fatePoolWatcher = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf); ThreadPools.watchCriticalScheduledTask(fatePoolWatcher.schedule(() -> { From c71692985aa2d04df2a57840a731b8e8a69bc336 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Tue, 21 Nov 2023 20:32:59 -0500 Subject: [PATCH 05/12] fix build --- core/src/main/java/org/apache/accumulo/core/fate/Fate.java | 4 ++++ 1 file changed, 4 insertions(+) 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 459d5cd1eb3..7b049986085 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 @@ -90,6 +90,8 @@ public enum TxInfo { // TODO add a task that periodically looks for fate task reserved by dead instances, only run in // partition zero private class LockCleaner implements Runnable { + + @Override public void run() { while (keepRunning.get()) { var partitionData = partitionDataSupplier.get(); @@ -116,6 +118,7 @@ private Retry newRetry() { .logInterval(3, MINUTES).createRetry(); } + @Override public void run() { try { @@ -131,6 +134,7 @@ public void run() { // attempt to avoid some unnecessary work caused by multiple processes trying to reserve // the same fate transaction. workQueue.clear(); + queuedWork.clear(); } lastPartitionData = partitionData; From 21b8ac9f430147cb83a3f4f56a10b3db4d3f3525 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 13:59:23 -0500 Subject: [PATCH 06/12] align names with #4017 --- .../apache/accumulo/core/fate/AdminUtil.java | 42 +++++++++---------- .../org/apache/accumulo/core/fate/Fate.java | 28 ++++++------- .../fate/{FatesStore.java => FateStore.java} | 8 ++-- ...FatesStore.java => ReadOnlyFateStore.java} | 6 +-- .../accumulo/core/fate/ZooFatesStore.java | 16 +++---- .../accumulo/core/logging/FateLogger.java | 28 ++++++------- .../apache/accumulo/server/util/Admin.java | 16 +++---- .../util/fateCommand/FateSummaryReport.java | 4 +- .../util/fateCommand/SummaryReportTest.java | 4 +- .../util/fateCommand/TxnDetailsTest.java | 8 ++-- .../accumulo/manager/FateServiceHandler.java | 2 +- .../metrics/fate/FateMetricValues.java | 10 ++--- .../manager/metrics/fate/FateMetrics.java | 23 +++++----- .../manager/upgrade/UpgradeCoordinator.java | 4 +- .../accumulo/test/fate/zookeeper/FateIT.java | 14 +++---- .../zookeeper/MultipleFateInstancesIT.java | 6 +-- 16 files changed, 109 insertions(+), 110 deletions(-) rename core/src/main/java/org/apache/accumulo/core/fate/{FatesStore.java => FateStore.java} (93%) rename core/src/main/java/org/apache/accumulo/core/fate/{ReadOnlyFatesStore.java => ReadOnlyFateStore.java} (96%) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index 369049de5b3..4b5cfc5b5bc 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -70,14 +70,14 @@ public AdminUtil(boolean exitOnError) { public static class TransactionStatus { private final long txid; - private final ReadOnlyFatesStore.FateStatus status; + private final ReadOnlyFateStore.FateStatus status; private final String txName; private final List hlocks; private final List wlocks; private final String top; private final long timeCreated; - private TransactionStatus(Long tid, ReadOnlyFatesStore.FateStatus status, String txName, + private TransactionStatus(Long tid, ReadOnlyFateStore.FateStatus status, String txName, List hlocks, List wlocks, String top, Long timeCreated) { this.txid = tid; @@ -98,7 +98,7 @@ public String getTxid() { return FastFormat.toHexString(txid); } - public ReadOnlyFatesStore.FateStatus getStatus() { + public ReadOnlyFateStore.FateStatus getStatus() { return status; } @@ -209,15 +209,15 @@ public Map> getDanglingWaitingLocks() { /** * Returns a list of the FATE transactions, optionally filtered by transaction id and status. This * method does not process lock information, if lock information is desired, use - * {@link #getStatus(ReadOnlyFatesStore, ZooReader, ServiceLockPath, Set, EnumSet)} + * {@link #getStatus(ReadOnlyFateStore, ZooReader, ServiceLockPath, Set, EnumSet)} * * @param zs read-only zoostore * @param filterTxid filter results to include for provided transaction ids. * @param filterStatus filter results to include only provided status types * @return list of FATE transactions that match filter criteria */ - public List getTransactionStatus(ReadOnlyFatesStore zs, - Set filterTxid, EnumSet filterStatus) { + public List getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, + EnumSet filterStatus) { FateStatus status = getTransactionStatus(zs, filterTxid, filterStatus, Collections.>emptyMap(), Collections.>emptyMap()); @@ -238,9 +238,9 @@ public List getTransactionStatus(ReadOnlyFatesStore zs, * @throws KeeperException if zookeeper exception occurs * @throws InterruptedException if process is interrupted. */ - public FateStatus getStatus(ReadOnlyFatesStore zs, ZooReader zk, + public FateStatus getStatus(ReadOnlyFateStore zs, ZooReader zk, ServiceLock.ServiceLockPath lockPath, Set filterTxid, - EnumSet filterStatus) + EnumSet filterStatus) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); Map> waitingLocks = new HashMap<>(); @@ -332,8 +332,8 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, * @param waitingLocks populated list of locks held by transaction - or an empty map if none. * @return current fate and lock status */ - private FateStatus getTransactionStatus(ReadOnlyFatesStore zs, Set filterTxid, - EnumSet filterStatus, Map> heldLocks, + private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, + EnumSet filterStatus, Map> heldLocks, Map> waitingLocks) { List transactions = zs.list(); @@ -363,7 +363,7 @@ private FateStatus getTransactionStatus(ReadOnlyFatesStore zs, Set filt top = repo.getName(); } - ReadOnlyFatesStore.FateStatus status = opStore.getStatus(); + ReadOnlyFateStore.FateStatus status = opStore.getStatus(); long timeCreated = opStore.timeCreated(); @@ -376,8 +376,8 @@ private FateStatus getTransactionStatus(ReadOnlyFatesStore zs, Set filt } - private boolean includeByStatus(ReadOnlyFatesStore.FateStatus status, - EnumSet filterStatus) { + private boolean includeByStatus(ReadOnlyFateStore.FateStatus status, + EnumSet filterStatus) { return (filterStatus == null) || filterStatus.contains(status); } @@ -385,14 +385,14 @@ private boolean includeByTxid(Long tid, Set filterTxid) { return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid); } - public void printAll(ReadOnlyFatesStore zs, ZooReader zk, + public void printAll(ReadOnlyFateStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath) throws KeeperException, InterruptedException { print(zs, zk, tableLocksPath, new Formatter(System.out), null, null); } - public void print(ReadOnlyFatesStore zs, ZooReader zk, + public void print(ReadOnlyFateStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set filterTxid, - EnumSet filterStatus) + EnumSet filterStatus) throws KeeperException, InterruptedException { FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus); @@ -417,7 +417,7 @@ public void print(ReadOnlyFatesStore zs, ZooReader zk, } } - public boolean prepDelete(FatesStore zs, ZooReaderWriter zk, ServiceLockPath path, + public boolean prepDelete(FateStore zs, ZooReaderWriter zk, ServiceLockPath path, String txidStr) { // TODO do not need global lock now // TODO need way to see what process holds a reservation @@ -435,7 +435,7 @@ public boolean prepDelete(FatesStore zs, ZooReaderWriter zk, ServiceLockPath boolean state = false; var opStore = zs.reserve(txid); - ReadOnlyFatesStore.FateStatus ts = opStore.getStatus(); + ReadOnlyFateStore.FateStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -457,7 +457,7 @@ public boolean prepDelete(FatesStore zs, ZooReaderWriter zk, ServiceLockPath return state; } - public boolean prepFail(FatesStore zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath, + public boolean prepFail(FateStore zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath, String txidStr) { if (!checkGlobalLock(zk, zLockManagerPath)) { return false; @@ -472,7 +472,7 @@ public boolean prepFail(FatesStore zs, ZooReaderWriter zk, ServiceLockPath zL } boolean state = false; var opStore = zs.reserve(txid); - ReadOnlyFatesStore.FateStatus ts = opStore.getStatus(); + ReadOnlyFateStore.FateStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -482,7 +482,7 @@ public boolean prepFail(FatesStore zs, ZooReaderWriter zk, ServiceLockPath zL case IN_PROGRESS: case NEW: System.out.printf("Failing transaction: %016x (%s)%n", txid, ts); - opStore.setStatus(ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS); + opStore.setStatus(ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS); state = true; break; 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 7b049986085..30b744b9568 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 @@ -22,13 +22,13 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.NEW; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUCCESSFUL; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.UNKNOWN; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.NEW; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUCCESSFUL; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.UNKNOWN; import static org.apache.accumulo.core.util.ShutdownUtil.isIOException; import java.util.Collections; @@ -49,7 +49,7 @@ import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus; import org.apache.accumulo.core.logging.FateLogger; import org.apache.accumulo.core.manager.PartitionData; import org.apache.accumulo.core.util.Retry; @@ -69,7 +69,7 @@ public class Fate { private static final Logger log = LoggerFactory.getLogger(Fate.class); private final Logger runnerLog = LoggerFactory.getLogger(TransactionRunner.class); - private final FatesStore store; + private final FateStore store; private final T environment; private final ScheduledThreadPoolExecutor fatePoolWatcher; private final ExecutorService executor; @@ -201,7 +201,7 @@ private class TransactionRunner implements Runnable { public void run() { while (keepRunning.get()) { long deferTime = 0; - FatesStore.FateStore opStore = null; + FateStore.FateTxStore opStore = null; try { var unreservedTid = workQueue.poll(100, MILLISECONDS); if (unreservedTid == null) { @@ -300,7 +300,7 @@ private void blockIfHadoopShutdown(long tid, Exception e) { } } - private void transitionToFailed(FatesStore.FateStore opStore, Exception e) { + private void transitionToFailed(FateStore.FateTxStore opStore, Exception e) { String tidStr = FateTxId.formatTid(opStore.getID()); final String msg = "Failed to execute Repo " + tidStr; // Certain FATE ops that throw exceptions don't need to be propagated up to the Monitor @@ -317,7 +317,7 @@ private void transitionToFailed(FatesStore.FateStore opStore, Exception e) { log.info("Updated status for Repo with {} to FAILED_IN_PROGRESS", tidStr); } - private void processFailed(FatesStore.FateStore opStore, Repo op) { + private void processFailed(FateStore.FateTxStore opStore, Repo op) { while (op != null) { undo(opStore.getID(), op); @@ -329,7 +329,7 @@ private void processFailed(FatesStore.FateStore opStore, Repo op) { doCleanUp(opStore); } - private void doCleanUp(FatesStore.FateStore opStore) { + private void doCleanUp(FateStore.FateTxStore opStore) { Boolean autoClean = (Boolean) opStore.getTransactionInfo(TxInfo.AUTO_CLEAN); if (autoClean != null && autoClean) { opStore.delete(); @@ -357,7 +357,7 @@ private void undo(long tid, Repo op) { * * @param toLogStrFunc A function that converts Repo to Strings that are suitable for logging */ - public Fate(T environment, FatesStore store, Function,String> toLogStrFunc, + public Fate(T environment, FateStore store, Function,String> toLogStrFunc, Supplier partitionDataSupplier, AccumuloConfiguration conf) { this.store = FateLogger.wrap(store, toLogStrFunc); this.environment = environment; diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java similarity index 93% rename from core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java rename to core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 4d4a8fbf8ae..6fb18374ee2 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/FatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java @@ -32,7 +32,7 @@ * as each step successfully completes. If a step fails, the stack can be unwound, undoing each * operation. */ -public interface FatesStore extends ReadOnlyFatesStore { +public interface FateStore extends ReadOnlyFateStore { /** * Create a new transaction id @@ -44,7 +44,7 @@ public interface FatesStore extends ReadOnlyFatesStore { /** * An interface that allows read/write access to the data related to a single fate operation. */ - interface FateStore extends ReadOnlyFateStore { + interface FateTxStore extends ReadOnlyFateTxStore { @Override Repo top(); @@ -99,7 +99,7 @@ interface FateStore extends ReadOnlyFateStore { * @param tid transaction id * @return true if reserved by this call, false if already reserved */ - Optional> tryReserve(long tid); + Optional> tryReserve(long tid); /** * Reserve the specific tid. @@ -108,6 +108,6 @@ interface FateStore extends ReadOnlyFateStore { * instance will be operating on that transaction id. * */ - FateStore reserve(long tid); + FateTxStore reserve(long tid); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java similarity index 96% rename from core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java rename to core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java index d25e3c552ac..076cf5eab31 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyFateStore.java @@ -31,7 +31,7 @@ * A transaction consists of a number of operations. Instances of this class may check on the queue * of outstanding transactions but may neither modify them nor create new ones. */ -public interface ReadOnlyFatesStore { +public interface ReadOnlyFateStore { /** * Possible operational status codes. Serialized by name within stores. @@ -53,7 +53,7 @@ enum FateStatus { SUBMITTED } - interface ReadOnlyFateStore { + interface ReadOnlyFateTxStore { /** * Get the current operation for the given transaction id. * @@ -106,7 +106,7 @@ interface ReadOnlyFateStore { long getID(); } - ReadOnlyFateStore read(long tid); + ReadOnlyFateTxStore read(long tid); /** * list all transaction ids in store. diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java index ad4be67db81..c709c721e50 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java @@ -65,7 +65,7 @@ //TODO use zoocache? - ACCUMULO-1297 //TODO handle zookeeper being down gracefully - ACCUMULO-1297 -public class ZooFatesStore implements FatesStore { +public class ZooFatesStore implements FateStore { private static final Logger log = LoggerFactory.getLogger(ZooFatesStore.class); private String path; @@ -183,7 +183,7 @@ public long create() { } @Override - public FateStore reserve(long tid) { + public FateTxStore reserve(long tid) { var retry = Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS) .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry(); @@ -210,7 +210,7 @@ public FateStore reserve(long tid) { * @return true if reserved by this call, false if already reserved */ @Override - public Optional> tryReserve(long tid) { + public Optional> tryReserve(long tid) { // uniquely identify this attempt to reserve the fate operation data var uuid = UUID.randomUUID(); @@ -231,7 +231,7 @@ public Optional> tryReserve(long tid) { if (newValue != null) { // clear any deferment if it exists, it may or may not be set when its unreserved defered.remove(tid); - return Optional.of(new FateStoreImpl(tid, uuid)); + return Optional.of(new FateTxStoreImpl(tid, uuid)); } else { return Optional.empty(); } @@ -243,7 +243,7 @@ public Optional> tryReserve(long tid) { } } - private class ReadOnlyFateStoreImpl implements ReadOnlyFateStore { + private class ReadOnlyFateStoreImpl implements ReadOnlyFateTxStore { private static final int RETRIES = 10; protected final long tid; @@ -435,14 +435,14 @@ protected void checkState(boolean unreserving) {} } - private class FateStoreImpl extends ReadOnlyFateStoreImpl implements FateStore { + private class FateTxStoreImpl extends ReadOnlyFateStoreImpl implements FateTxStore { private boolean reserved = true; private boolean deleted = false; private final UUID uuid; - protected FateStoreImpl(long tid, UUID uuid) { + protected FateTxStoreImpl(long tid, UUID uuid) { super(tid); this.uuid = Objects.requireNonNull(uuid); } @@ -578,7 +578,7 @@ protected void checkState(boolean unreserving) { } @Override - public ReadOnlyFateStore read(long tid) { + public ReadOnlyFateTxStore read(long tid) { return new ReadOnlyFateStoreImpl(tid); } 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 01383f8155e..7a4ce8a052e 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 @@ -28,8 +28,8 @@ import java.util.function.Function; import org.apache.accumulo.core.fate.Fate; -import org.apache.accumulo.core.fate.FatesStore; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.FateStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; @@ -44,12 +44,12 @@ public class FateLogger { // reproducible problems with FATE transactions. private static final Logger storeLog = LoggerFactory.getLogger(PREFIX + "store"); - private static class LoggingFateStore implements FatesStore.FateStore { + private static class LoggingFateStore implements FateStore.FateTxStore { - private final FatesStore.FateStore wrapped; + private final FateStore.FateTxStore wrapped; private final Function,String> toLogString; - private LoggingFateStore(FatesStore.FateStore wrapped, + private LoggingFateStore(FateStore.FateTxStore wrapped, Function,String> toLogString) { this.wrapped = wrapped; this.toLogString = toLogString; @@ -61,13 +61,13 @@ public List> getStack() { } @Override - public ReadOnlyFatesStore.FateStatus getStatus() { + public ReadOnlyFateStore.FateStatus getStatus() { return wrapped.getStatus(); } @Override - public ReadOnlyFatesStore.FateStatus - waitForStatusChange(EnumSet expected) { + public ReadOnlyFateStore.FateStatus + waitForStatusChange(EnumSet expected) { return wrapped.waitForStatusChange(expected); } @@ -108,7 +108,7 @@ public void pop() { } @Override - public void setStatus(ReadOnlyFatesStore.FateStatus status) { + public void setStatus(ReadOnlyFateStore.FateStatus status) { wrapped.setStatus(status); if (storeLog.isTraceEnabled()) { storeLog.trace("{} setStatus to {}", formatTid(wrapped.getID()), status); @@ -137,22 +137,22 @@ public void unreserve(long deferTime) { } } - public static FatesStore wrap(FatesStore store, Function,String> toLogString) { + public static FateStore wrap(FateStore store, Function,String> toLogString) { // only logging operations that change the persisted data, not operations that only read data - return new FatesStore<>() { + return new FateStore<>() { @Override - public FateStore reserve(long tid) { + public FateTxStore reserve(long tid) { return new LoggingFateStore<>(store.reserve(tid), toLogString); } @Override - public Optional> tryReserve(long tid) { + public Optional> tryReserve(long tid) { return store.tryReserve(tid).map(fos -> new LoggingFateStore<>(fos, toLogString)); } @Override - public ReadOnlyFateStore read(long tid) { + public ReadOnlyFateTxStore read(long tid) { return store.read(tid); } 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 9e1eda3f3aa..b5ef10b2788 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 @@ -55,7 +55,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.FateTxId; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; @@ -783,7 +783,7 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps if (fateOpsCommand.print) { final Set sortedTxs = new TreeSet<>(); fateOpsCommand.txList.forEach(s -> sortedTxs.add(parseTidFromUserInput(s))); - EnumSet statusFilter = + EnumSet statusFilter = getCmdLineStatusFilters(fateOpsCommand.states); admin.print(zs, zk, zTableLocksPath, new Formatter(System.out), sortedTxs, statusFilter); // print line break at the end @@ -836,7 +836,7 @@ private boolean cancelFateOperation(ClientContext context, long txid) throws Acc } private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUtil admin, - ReadOnlyFatesStore zs, ServiceLock.ServiceLockPath tableLocksPath) + ReadOnlyFateStore zs, ServiceLock.ServiceLockPath tableLocksPath) throws InterruptedException, AccumuloException, AccumuloSecurityException, KeeperException { ZooReaderWriter zk = context.getZooReaderWriter(); @@ -855,7 +855,7 @@ private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUti } }); - EnumSet statusFilter = getCmdLineStatusFilters(cmd.states); + EnumSet statusFilter = getCmdLineStatusFilters(cmd.states); FateSummaryReport report = new FateSummaryReport(idsToNameMap, statusFilter); @@ -882,12 +882,12 @@ private void printLines(List lines) { * * @return a set of status filters, or an empty set if none provides */ - private EnumSet getCmdLineStatusFilters(List states) { - EnumSet statusFilter = null; + private EnumSet getCmdLineStatusFilters(List states) { + EnumSet statusFilter = null; if (!states.isEmpty()) { - statusFilter = EnumSet.noneOf(ReadOnlyFatesStore.FateStatus.class); + statusFilter = EnumSet.noneOf(ReadOnlyFateStore.FateStatus.class); for (String element : states) { - statusFilter.add(ReadOnlyFatesStore.FateStatus.valueOf(element)); + statusFilter.add(ReadOnlyFateStore.FateStatus.valueOf(element)); } } return statusFilter; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java index f6550b31882..ee64329208d 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java @@ -33,7 +33,7 @@ import java.util.TreeSet; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -55,7 +55,7 @@ public class FateSummaryReport { private final transient Map idsToNameMap; public FateSummaryReport(Map idsToNameMap, - EnumSet statusFilter) { + EnumSet statusFilter) { this.idsToNameMap = idsToNameMap; if (statusFilter != null) { statusFilter.forEach(f -> this.statusFilterNames.add(f.name())); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java index 5ab3f6098c7..ebede860e3f 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,7 +67,7 @@ public void noTablenameReport() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn(null).anyTimes(); expect(status1.getTxName()).andReturn(null).anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java index 8aa27783e86..102f59e71cc 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java @@ -33,7 +33,7 @@ import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +50,7 @@ void orderingByDuration() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn("step1").anyTimes(); expect(status1.getTxName()).andReturn("runningTx1").anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); @@ -59,7 +59,7 @@ void orderingByDuration() { AdminUtil.TransactionStatus status2 = createMock(AdminUtil.TransactionStatus.class); expect(status2.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(7)).anyTimes(); - expect(status2.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status2.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status2.getTop()).andReturn("step2").anyTimes(); expect(status2.getTxName()).andReturn("runningTx2").anyTimes(); expect(status2.getTxid()).andReturn("123456789").anyTimes(); @@ -93,7 +93,7 @@ public void lockTest() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyFatesStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn("step1").anyTimes(); expect(status1.getTxName()).andReturn("runningTx").anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index 1a5debb27ba..f31e18a59a7 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -68,7 +68,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TRange; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus; import org.apache.accumulo.core.manager.thrift.BulkImportState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java index 08372aa9e63..5e007935fe8 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java @@ -24,7 +24,7 @@ import java.util.TreeMap; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -101,7 +101,7 @@ Map getOpTypeCounters() { * @return the current FATE metric values. */ public static FateMetricValues getFromZooKeeper(final ServerContext context, - final String fateRootPath, final ReadOnlyFatesStore zooStore) { + final String fateRootPath, final ReadOnlyFateStore zooStore) { FateMetricValues.Builder builder = FateMetricValues.builder(); @@ -116,7 +116,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, // states are enumerated - create new map with counts initialized to 0. Map states = new TreeMap<>(); - for (ReadOnlyFatesStore.FateStatus t : ReadOnlyFatesStore.FateStatus.values()) { + for (ReadOnlyFateStore.FateStatus t : ReadOnlyFateStore.FateStatus.values()) { states.put(t.name(), 0L); } @@ -132,7 +132,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, states.merge(stateName, 1L, Long::sum); // incr count for op type for for in_progress transactions. - if (ReadOnlyFatesStore.FateStatus.IN_PROGRESS.equals(tx.getStatus())) { + if (ReadOnlyFateStore.FateStatus.IN_PROGRESS.equals(tx.getStatus())) { String opType = tx.getTxName(); if (opType == null || opType.isEmpty()) { opType = "UNKNOWN"; @@ -189,7 +189,7 @@ static class Builder { // states are enumerated - create new map with counts initialized to 0. txStateCounters = new TreeMap<>(); - for (ReadOnlyFatesStore.FateStatus t : ReadOnlyFatesStore.FateStatus.values()) { + for (ReadOnlyFateStore.FateStatus t : ReadOnlyFateStore.FateStatus.values()) { txStateCounters.put(t.name(), 0L); } 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 7558cb68b01..546414f6ed1 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 @@ -25,7 +25,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.metrics.MetricsUtil; @@ -49,7 +49,7 @@ public class FateMetrics implements MetricsProducer { private static final String OP_TYPE_TAG = "op.type"; private final ServerContext context; - private final ReadOnlyFatesStore zooStore; + private final ReadOnlyFateStore zooStore; private final String fateRootPath; private final long refreshDelay; @@ -93,7 +93,7 @@ private void update() { fateErrorsGauge.set(metricValues.getZkConnectionErrors()); for (Entry vals : metricValues.getTxStateCounters().entrySet()) { - switch (ReadOnlyFatesStore.FateStatus.valueOf(vals.getKey())) { + switch (ReadOnlyFateStore.FateStatus.valueOf(vals.getKey())) { case NEW: newTxGauge.set(vals.getValue()); break; @@ -134,22 +134,21 @@ public void registerMetrics(final MeterRegistry registry) { fateErrorsGauge = registry.gauge(METRICS_FATE_ERRORS, Tags.concat(MetricsUtil.getCommonTags(), "type", "zk.connection"), new AtomicLong(0)); newTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyFatesStore.FateStatus.NEW.name().toLowerCase()), new AtomicLong(0)); + ReadOnlyFateStore.FateStatus.NEW.name().toLowerCase()), new AtomicLong(0)); submittedTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFatesStore.FateStatus.SUBMITTED.name().toLowerCase()), new AtomicLong(0)); - inProgressTxGauge = - registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyFatesStore.FateStatus.IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.FateStatus.SUBMITTED.name().toLowerCase()), new AtomicLong(0)); + inProgressTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), + "state", ReadOnlyFateStore.FateStatus.IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); failedInProgressTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS.name().toLowerCase()), + ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); failedTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFatesStore.FateStatus.FAILED.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.FateStatus.FAILED.name().toLowerCase()), new AtomicLong(0)); successfulTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFatesStore.FateStatus.SUCCESSFUL.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.FateStatus.SUCCESSFUL.name().toLowerCase()), new AtomicLong(0)); unknownTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFatesStore.FateStatus.UNKNOWN.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.FateStatus.UNKNOWN.name().toLowerCase()), new AtomicLong(0)); update(); 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 f7699c96442..596a9610ee7 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 @@ -32,7 +32,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.util.threads.ThreadPools; @@ -268,7 +268,7 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - final ReadOnlyFatesStore fate = new ZooFatesStore<>( + final ReadOnlyFateStore fate = new ZooFatesStore<>( context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter(), null); // TODO // maybe // have diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index 74c7dc6db44..b78f2fe77ce 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -19,12 +19,12 @@ package org.apache.accumulo.test.fate.zookeeper; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.FAILED_IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.NEW; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus.SUCCESSFUL; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.NEW; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUCCESSFUL; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateTxId; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore.FateStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java index dafb59753fd..819d18854a8 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java @@ -31,7 +31,7 @@ import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.fate.Fate; -import org.apache.accumulo.core.fate.ReadOnlyFatesStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; @@ -95,12 +95,12 @@ public void testZooStores() throws Exception { if (txid % 2 == 0) { var rfo = zooFatesStore1.reserve(txid); assertTrue(zooFatesStore2.tryReserve(txid).isEmpty()); - rfo.setStatus(ReadOnlyFatesStore.FateStatus.SUBMITTED); + rfo.setStatus(ReadOnlyFateStore.FateStatus.SUBMITTED); rfo.unreserve(0); } else { var rfo = zooFatesStore2.reserve(txid); assertTrue(zooFatesStore1.tryReserve(txid).isEmpty()); - rfo.setStatus(ReadOnlyFatesStore.FateStatus.SUBMITTED); + rfo.setStatus(ReadOnlyFateStore.FateStatus.SUBMITTED); rfo.unreserve(0); } } From f5097643bae4563f5147d4a967aeaaed9cbd07a1 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 14:15:03 -0500 Subject: [PATCH 07/12] Align name with elasticity branch --- .../apache/accumulo/core/fate/AdminUtil.java | 26 +++++++++---------- .../org/apache/accumulo/core/fate/Fate.java | 25 +++++++++--------- .../apache/accumulo/core/fate/FateStore.java | 2 +- .../accumulo/core/fate/ReadOnlyFateStore.java | 7 ++--- .../accumulo/core/fate/ZooFatesStore.java | 26 +++++++++---------- .../accumulo/core/logging/FateLogger.java | 8 +++--- .../apache/accumulo/server/util/Admin.java | 12 ++++----- .../util/fateCommand/FateSummaryReport.java | 2 +- .../util/fateCommand/SummaryReportTest.java | 2 +- .../util/fateCommand/TxnDetailsTest.java | 6 ++--- .../accumulo/manager/FateServiceHandler.java | 6 ++--- .../metrics/fate/FateMetricValues.java | 6 ++--- .../manager/metrics/fate/FateMetrics.java | 23 ++++++++-------- .../accumulo/test/fate/zookeeper/FateIT.java | 24 ++++++++--------- .../zookeeper/MultipleFateInstancesIT.java | 4 +-- 15 files changed, 90 insertions(+), 89 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index 4b5cfc5b5bc..bd0dc8cbbf9 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -70,14 +70,14 @@ public AdminUtil(boolean exitOnError) { public static class TransactionStatus { private final long txid; - private final ReadOnlyFateStore.FateStatus status; + private final ReadOnlyFateStore.TStatus status; private final String txName; private final List hlocks; private final List wlocks; private final String top; private final long timeCreated; - private TransactionStatus(Long tid, ReadOnlyFateStore.FateStatus status, String txName, + private TransactionStatus(Long tid, ReadOnlyFateStore.TStatus status, String txName, List hlocks, List wlocks, String top, Long timeCreated) { this.txid = tid; @@ -98,7 +98,7 @@ public String getTxid() { return FastFormat.toHexString(txid); } - public ReadOnlyFateStore.FateStatus getStatus() { + public ReadOnlyFateStore.TStatus getStatus() { return status; } @@ -217,7 +217,7 @@ public Map> getDanglingWaitingLocks() { * @return list of FATE transactions that match filter criteria */ public List getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, - EnumSet filterStatus) { + EnumSet filterStatus) { FateStatus status = getTransactionStatus(zs, filterTxid, filterStatus, Collections.>emptyMap(), Collections.>emptyMap()); @@ -240,7 +240,7 @@ public List getTransactionStatus(ReadOnlyFateStore zs, Set */ public FateStatus getStatus(ReadOnlyFateStore zs, ZooReader zk, ServiceLock.ServiceLockPath lockPath, Set filterTxid, - EnumSet filterStatus) + EnumSet filterStatus) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); Map> waitingLocks = new HashMap<>(); @@ -333,7 +333,7 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, * @return current fate and lock status */ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, - EnumSet filterStatus, Map> heldLocks, + EnumSet filterStatus, Map> heldLocks, Map> waitingLocks) { List transactions = zs.list(); @@ -363,7 +363,7 @@ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filte top = repo.getName(); } - ReadOnlyFateStore.FateStatus status = opStore.getStatus(); + ReadOnlyFateStore.TStatus status = opStore.getStatus(); long timeCreated = opStore.timeCreated(); @@ -376,8 +376,8 @@ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filte } - private boolean includeByStatus(ReadOnlyFateStore.FateStatus status, - EnumSet filterStatus) { + private boolean includeByStatus(ReadOnlyFateStore.TStatus status, + EnumSet filterStatus) { return (filterStatus == null) || filterStatus.contains(status); } @@ -392,7 +392,7 @@ public void printAll(ReadOnlyFateStore zs, ZooReader zk, public void print(ReadOnlyFateStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set filterTxid, - EnumSet filterStatus) + EnumSet filterStatus) throws KeeperException, InterruptedException { FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus); @@ -435,7 +435,7 @@ public boolean prepDelete(FateStore zs, ZooReaderWriter zk, ServiceLockPath p boolean state = false; var opStore = zs.reserve(txid); - ReadOnlyFateStore.FateStatus ts = opStore.getStatus(); + ReadOnlyFateStore.TStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -472,7 +472,7 @@ public boolean prepFail(FateStore zs, ZooReaderWriter zk, ServiceLockPath zLo } boolean state = false; var opStore = zs.reserve(txid); - ReadOnlyFateStore.FateStatus ts = opStore.getStatus(); + ReadOnlyFateStore.TStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -482,7 +482,7 @@ public boolean prepFail(FateStore zs, ZooReaderWriter zk, ServiceLockPath zLo case IN_PROGRESS: case NEW: System.out.printf("Failing transaction: %016x (%s)%n", txid, ts); - opStore.setStatus(ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS); + opStore.setStatus(ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS); state = true; break; 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 30b744b9568..2c908c0b4b3 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 @@ -22,13 +22,13 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.NEW; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUCCESSFUL; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.UNKNOWN; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.FAILED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.NEW; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.SUCCESSFUL; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.UNKNOWN; import static org.apache.accumulo.core.util.ShutdownUtil.isIOException; import java.util.Collections; @@ -49,7 +49,7 @@ import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.logging.FateLogger; import org.apache.accumulo.core.manager.PartitionData; import org.apache.accumulo.core.util.Retry; @@ -74,8 +74,7 @@ public class Fate { private final ScheduledThreadPoolExecutor fatePoolWatcher; private final ExecutorService executor; - private static final EnumSet FINISHED_STATES = - EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); + private static final EnumSet FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN); private final AtomicBoolean keepRunning = new AtomicBoolean(true); private final Supplier partitionDataSupplier; @@ -214,7 +213,7 @@ public void run() { } else { continue; } - FateStatus status = opStore.getStatus(); + TStatus status = opStore.getStatus(); Repo op = opStore.top(); if (status == FAILED_IN_PROGRESS) { processFailed(opStore, op); @@ -436,7 +435,7 @@ public void seedTransaction(String txName, long tid, Repo repo, boolean autoC } // check on the transaction - public FateStatus waitForCompletion(long tid) { + public TStatus waitForCompletion(long tid) { return store.read(tid).waitForStatusChange(FINISHED_STATES); } @@ -454,7 +453,7 @@ public boolean cancel(long tid) { if (optionalOpStore.isPresent()) { var opStore = optionalOpStore.orElseThrow(); try { - FateStatus status = opStore.getStatus(); + TStatus status = opStore.getStatus(); log.info("status is: {}", status); if (status == NEW || status == SUBMITTED) { opStore.setTransactionInfo(TxInfo.EXCEPTION, new TApplicationException( 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 6fb18374ee2..33acb2d56a6 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 @@ -66,7 +66,7 @@ interface FateTxStore extends ReadOnlyFateTxStore { * * @param status execution status */ - void setStatus(FateStatus status); + void setStatus(TStatus status); /** * Set transaction-specific information. 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 076cf5eab31..b4b68a1cf4a 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 @@ -36,7 +36,8 @@ public interface ReadOnlyFateStore { /** * Possible operational status codes. Serialized by name within stores. */ - enum FateStatus { + // TODO rename to FateTxStatus + enum TStatus { /** Unseeded transaction */ NEW, /** Transaction that is executing */ @@ -76,7 +77,7 @@ interface ReadOnlyFateTxStore { * * @return execution status */ - FateStatus getStatus(); + TStatus getStatus(); /** * Wait for the status of a transaction to change @@ -85,7 +86,7 @@ interface ReadOnlyFateTxStore { * be null. * @return execution status. */ - FateStatus waitForStatusChange(EnumSet expected); + TStatus waitForStatusChange(EnumSet expected); /** * Retrieve transaction-specific information. diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java index c709c721e50..e333d5f7ce4 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java @@ -79,18 +79,18 @@ public class ZooFatesStore implements FateStore { // TODO this was hastily written code for serialization. Should it use json? It needs better // validation of data being serialized (like validate status, lock, and uuid). private static class NodeValue { - final FateStatus status; + final TStatus status; final String lock; final String uuid; private NodeValue(byte[] serializedData) { var fields = new String(serializedData, UTF_8).split(":", 3); - this.status = FateStatus.valueOf(fields[0]); + this.status = TStatus.valueOf(fields[0]); this.lock = fields[1]; this.uuid = fields[2]; } - private NodeValue(FateStatus status, String lock, String uuid) { + private NodeValue(TStatus status, String lock, String uuid) { if (lock.isEmpty() && !uuid.isEmpty() || uuid.isEmpty() && !lock.isEmpty()) { throw new IllegalArgumentException( "For lock and uuid expect neither is empty or both are empty. lock:'" + lock @@ -171,7 +171,7 @@ public long create() { try { // looking at the code for SecureRandom, it appears to be thread safe long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL; - zk.putPersistentData(getTXPath(tid), new NodeValue(FateStatus.NEW, "", "").serialize(), + zk.putPersistentData(getTXPath(tid), new NodeValue(TStatus.NEW, "", "").serialize(), NodeExistsPolicy.FAIL); return tid; } catch (NodeExistsException nee) { @@ -304,25 +304,25 @@ String findTop(String txpath) throws KeeperException, InterruptedException { return max; } - private FateStatus _getStatus() { + private TStatus _getStatus() { checkState(false); try { return new NodeValue(zk.getData(getTXPath(tid))).status; } catch (NoNodeException nne) { - return FateStatus.UNKNOWN; + return TStatus.UNKNOWN; } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } } @Override - public FateStatus getStatus() { + public TStatus getStatus() { checkState(false); return _getStatus(); } @Override - public FateStatus waitForStatusChange(EnumSet expected) { + public TStatus waitForStatusChange(EnumSet expected) { checkState(false); // 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 @@ -334,7 +334,7 @@ public FateStatus waitForStatusChange(EnumSet expected) { .logInterval(3, MINUTES).createRetry(); while (true) { - FateStatus status = _getStatus(); + TStatus status = _getStatus(); if (expected.contains(status)) { retry.logCompletion(log, "Waiting on status change for " + FateTxId.formatTid(tid) + " expected:" + expected + " status:" + status); @@ -481,7 +481,7 @@ public void pop() { } @Override - public void setStatus(FateStatus status) { + public void setStatus(TStatus status) { checkState(false); try { zk.mutateExisting(getTXPath(tid), currentValue -> { @@ -606,9 +606,9 @@ public Iterator runnable(PartitionData partitionData) { var nodeVal = new NodeValue(zk.getData(path + "/" + txid)); var tid = parseTid(txid); if (!nodeVal.isReserved() - && (nodeVal.status == FateStatus.IN_PROGRESS - || nodeVal.status == FateStatus.FAILED_IN_PROGRESS - || nodeVal.status == FateStatus.SUBMITTED) + && (nodeVal.status == TStatus.IN_PROGRESS + || nodeVal.status == TStatus.FAILED_IN_PROGRESS + || nodeVal.status == TStatus.SUBMITTED) && tid % partitionData.getTotalInstances() == partitionData.getPartition()) { runnableTids.add(tid); } 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 7a4ce8a052e..5635bd11f81 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 @@ -61,13 +61,13 @@ public List> getStack() { } @Override - public ReadOnlyFateStore.FateStatus getStatus() { + public ReadOnlyFateStore.TStatus getStatus() { return wrapped.getStatus(); } @Override - public ReadOnlyFateStore.FateStatus - waitForStatusChange(EnumSet expected) { + public ReadOnlyFateStore.TStatus + waitForStatusChange(EnumSet expected) { return wrapped.waitForStatusChange(expected); } @@ -108,7 +108,7 @@ public void pop() { } @Override - public void setStatus(ReadOnlyFateStore.FateStatus status) { + public void setStatus(ReadOnlyFateStore.TStatus status) { wrapped.setStatus(status); if (storeLog.isTraceEnabled()) { storeLog.trace("{} setStatus to {}", formatTid(wrapped.getID()), status); 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 b5ef10b2788..0664f7eafef 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 @@ -783,7 +783,7 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps if (fateOpsCommand.print) { final Set sortedTxs = new TreeSet<>(); fateOpsCommand.txList.forEach(s -> sortedTxs.add(parseTidFromUserInput(s))); - EnumSet statusFilter = + EnumSet statusFilter = getCmdLineStatusFilters(fateOpsCommand.states); admin.print(zs, zk, zTableLocksPath, new Formatter(System.out), sortedTxs, statusFilter); // print line break at the end @@ -855,7 +855,7 @@ private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUti } }); - EnumSet statusFilter = getCmdLineStatusFilters(cmd.states); + EnumSet statusFilter = getCmdLineStatusFilters(cmd.states); FateSummaryReport report = new FateSummaryReport(idsToNameMap, statusFilter); @@ -882,12 +882,12 @@ private void printLines(List lines) { * * @return a set of status filters, or an empty set if none provides */ - private EnumSet getCmdLineStatusFilters(List states) { - EnumSet statusFilter = null; + private EnumSet getCmdLineStatusFilters(List states) { + EnumSet statusFilter = null; if (!states.isEmpty()) { - statusFilter = EnumSet.noneOf(ReadOnlyFateStore.FateStatus.class); + statusFilter = EnumSet.noneOf(ReadOnlyFateStore.TStatus.class); for (String element : states) { - statusFilter.add(ReadOnlyFateStore.FateStatus.valueOf(element)); + statusFilter.add(ReadOnlyFateStore.TStatus.valueOf(element)); } } return statusFilter; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java index ee64329208d..f99e36d704f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/fateCommand/FateSummaryReport.java @@ -55,7 +55,7 @@ public class FateSummaryReport { private final transient Map idsToNameMap; public FateSummaryReport(Map idsToNameMap, - EnumSet statusFilter) { + EnumSet statusFilter) { this.idsToNameMap = idsToNameMap; if (statusFilter != null) { statusFilter.forEach(f -> this.statusFilterNames.add(f.name())); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java index ebede860e3f..bed17e92c40 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java @@ -67,7 +67,7 @@ public void noTablenameReport() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFateStore.TStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn(null).anyTimes(); expect(status1.getTxName()).andReturn(null).anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java index 102f59e71cc..48e5c598453 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/TxnDetailsTest.java @@ -50,7 +50,7 @@ void orderingByDuration() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFateStore.TStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn("step1").anyTimes(); expect(status1.getTxName()).andReturn("runningTx1").anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); @@ -59,7 +59,7 @@ void orderingByDuration() { AdminUtil.TransactionStatus status2 = createMock(AdminUtil.TransactionStatus.class); expect(status2.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(7)).anyTimes(); - expect(status2.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status2.getStatus()).andReturn(ReadOnlyFateStore.TStatus.IN_PROGRESS).anyTimes(); expect(status2.getTop()).andReturn("step2").anyTimes(); expect(status2.getTxName()).andReturn("runningTx2").anyTimes(); expect(status2.getTxid()).andReturn("123456789").anyTimes(); @@ -93,7 +93,7 @@ public void lockTest() { AdminUtil.TransactionStatus status1 = createMock(AdminUtil.TransactionStatus.class); expect(status1.getTimeCreated()).andReturn(now - TimeUnit.DAYS.toMillis(1)).anyTimes(); - expect(status1.getStatus()).andReturn(ReadOnlyFateStore.FateStatus.IN_PROGRESS).anyTimes(); + expect(status1.getStatus()).andReturn(ReadOnlyFateStore.TStatus.IN_PROGRESS).anyTimes(); expect(status1.getTop()).andReturn("step1").anyTimes(); expect(status1.getTxName()).andReturn("runningTx").anyTimes(); expect(status1.getTxid()).andReturn("abcdabcd").anyTimes(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index f31e18a59a7..d04fc9455c1 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -68,7 +68,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TRange; -import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.manager.thrift.BulkImportState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; @@ -823,8 +823,8 @@ public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long o throws ThriftSecurityException, ThriftTableOperationException { authenticate(credentials); - FateStatus status = manager.fate().waitForCompletion(opid); - if (status == FateStatus.FAILED) { + TStatus status = manager.fate().waitForCompletion(opid); + if (status == TStatus.FAILED) { Exception e = manager.fate().getException(opid); if (e instanceof ThriftTableOperationException) { throw (ThriftTableOperationException) e; diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java index 5e007935fe8..e31ff65398e 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java @@ -116,7 +116,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, // states are enumerated - create new map with counts initialized to 0. Map states = new TreeMap<>(); - for (ReadOnlyFateStore.FateStatus t : ReadOnlyFateStore.FateStatus.values()) { + for (ReadOnlyFateStore.TStatus t : ReadOnlyFateStore.TStatus.values()) { states.put(t.name(), 0L); } @@ -132,7 +132,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, states.merge(stateName, 1L, Long::sum); // incr count for op type for for in_progress transactions. - if (ReadOnlyFateStore.FateStatus.IN_PROGRESS.equals(tx.getStatus())) { + if (ReadOnlyFateStore.TStatus.IN_PROGRESS.equals(tx.getStatus())) { String opType = tx.getTxName(); if (opType == null || opType.isEmpty()) { opType = "UNKNOWN"; @@ -189,7 +189,7 @@ static class Builder { // states are enumerated - create new map with counts initialized to 0. txStateCounters = new TreeMap<>(); - for (ReadOnlyFateStore.FateStatus t : ReadOnlyFateStore.FateStatus.values()) { + for (ReadOnlyFateStore.TStatus t : ReadOnlyFateStore.TStatus.values()) { txStateCounters.put(t.name(), 0L); } 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 546414f6ed1..11ce23147d2 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 @@ -93,7 +93,7 @@ private void update() { fateErrorsGauge.set(metricValues.getZkConnectionErrors()); for (Entry vals : metricValues.getTxStateCounters().entrySet()) { - switch (ReadOnlyFateStore.FateStatus.valueOf(vals.getKey())) { + switch (ReadOnlyFateStore.TStatus.valueOf(vals.getKey())) { case NEW: newTxGauge.set(vals.getValue()); break; @@ -134,21 +134,22 @@ public void registerMetrics(final MeterRegistry registry) { fateErrorsGauge = registry.gauge(METRICS_FATE_ERRORS, Tags.concat(MetricsUtil.getCommonTags(), "type", "zk.connection"), new AtomicLong(0)); newTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyFateStore.FateStatus.NEW.name().toLowerCase()), new AtomicLong(0)); + ReadOnlyFateStore.TStatus.NEW.name().toLowerCase()), new AtomicLong(0)); submittedTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFateStore.FateStatus.SUBMITTED.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.TStatus.SUBMITTED.name().toLowerCase()), new AtomicLong(0)); inProgressTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFateStore.FateStatus.IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); - failedInProgressTxGauge = registry.gauge(METRICS_FATE_TX, - Tags.concat(MetricsUtil.getCommonTags(), "state", - ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS.name().toLowerCase()), - new AtomicLong(0)); + "state", ReadOnlyFateStore.TStatus.IN_PROGRESS.name().toLowerCase()), new AtomicLong(0)); + failedInProgressTxGauge = + registry.gauge(METRICS_FATE_TX, + Tags.concat(MetricsUtil.getCommonTags(), "state", + ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS.name().toLowerCase()), + new AtomicLong(0)); failedTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFateStore.FateStatus.FAILED.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.TStatus.FAILED.name().toLowerCase()), new AtomicLong(0)); successfulTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFateStore.FateStatus.SUCCESSFUL.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.TStatus.SUCCESSFUL.name().toLowerCase()), new AtomicLong(0)); unknownTxGauge = registry.gauge(METRICS_FATE_TX, Tags.concat(MetricsUtil.getCommonTags(), - "state", ReadOnlyFateStore.FateStatus.UNKNOWN.name().toLowerCase()), new AtomicLong(0)); + "state", ReadOnlyFateStore.TStatus.UNKNOWN.name().toLowerCase()), new AtomicLong(0)); update(); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index b78f2fe77ce..b3adec7e468 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -19,12 +19,12 @@ package org.apache.accumulo.test.fate.zookeeper; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.FAILED_IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.IN_PROGRESS; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.NEW; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUBMITTED; -import static org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus.SUCCESSFUL; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.FAILED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.IN_PROGRESS; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.NEW; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.SUBMITTED; +import static org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus.SUCCESSFUL; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; @@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateTxId; -import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.ZooFatesStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; @@ -177,16 +177,16 @@ public void testTransactionStatus() throws Exception { finishCall = new CountDownLatch(1); long txid = fate.startTransaction(); - assertEquals(FateStatus.NEW, getTxStatus(zk, txid)); + assertEquals(TStatus.NEW, getTxStatus(zk, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); - assertEquals(FateStatus.SUBMITTED, getTxStatus(zk, txid)); + assertEquals(TStatus.SUBMITTED, getTxStatus(zk, txid)); // wait for call() to be called callStarted.await(); assertEquals(IN_PROGRESS, getTxStatus(zk, txid)); // tell the op to exit the method finishCall.countDown(); // Check that it transitions to SUCCESSFUL - FateStatus s = getTxStatus(zk, txid); + TStatus s = getTxStatus(zk, txid); while (s != SUCCESSFUL) { s = getTxStatus(zk, txid); Thread.sleep(10); @@ -343,11 +343,11 @@ private static void inCall() throws InterruptedException { * Get the status of the TX from ZK directly. Unable to call ZooStore.getStatus because this test * thread does not have the reservation (the FaTE thread does) */ - private static FateStatus getTxStatus(ZooReaderWriter zrw, long txid) + private static TStatus getTxStatus(ZooReaderWriter zrw, long txid) throws KeeperException, InterruptedException { zrw.sync(ZK_ROOT); String txdir = String.format("%s%s/tx_%016x", ZK_ROOT, Constants.ZFATE, txid); - return FateStatus.valueOf(new String(zrw.getData(txdir), UTF_8).split(":")[0]); + return TStatus.valueOf(new String(zrw.getData(txdir), UTF_8).split(":")[0]); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java index 819d18854a8..c9c75277977 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java @@ -95,12 +95,12 @@ public void testZooStores() throws Exception { if (txid % 2 == 0) { var rfo = zooFatesStore1.reserve(txid); assertTrue(zooFatesStore2.tryReserve(txid).isEmpty()); - rfo.setStatus(ReadOnlyFateStore.FateStatus.SUBMITTED); + rfo.setStatus(ReadOnlyFateStore.TStatus.SUBMITTED); rfo.unreserve(0); } else { var rfo = zooFatesStore2.reserve(txid); assertTrue(zooFatesStore1.tryReserve(txid).isEmpty()); - rfo.setStatus(ReadOnlyFateStore.FateStatus.SUBMITTED); + rfo.setStatus(ReadOnlyFateStore.TStatus.SUBMITTED); rfo.unreserve(0); } } From 2cf80e7b95575ee9f1ab9760f554596480cf1af3 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 14:21:04 -0500 Subject: [PATCH 08/12] Align name with elasticity branch --- .../{ZooFatesStore.java => ZooStore.java} | 8 ++-- .../apache/accumulo/server/util/Admin.java | 4 +- .../org/apache/accumulo/manager/Manager.java | 4 +- .../manager/metrics/fate/FateMetrics.java | 4 +- .../manager/upgrade/UpgradeCoordinator.java | 4 +- .../accumulo/test/fate/zookeeper/FateIT.java | 18 ++++---- .../zookeeper/MultipleFateInstancesIT.java | 42 +++++++++---------- .../test/functional/FateConcurrencyIT.java | 10 ++--- .../test/functional/FunctionalTestUtils.java | 5 +-- 9 files changed, 49 insertions(+), 50 deletions(-) rename core/src/main/java/org/apache/accumulo/core/fate/{ZooFatesStore.java => ZooStore.java} (98%) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java similarity index 98% rename from core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java rename to core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java index e333d5f7ce4..256d2d48918 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ZooFatesStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java @@ -65,9 +65,9 @@ //TODO use zoocache? - ACCUMULO-1297 //TODO handle zookeeper being down gracefully - ACCUMULO-1297 -public class ZooFatesStore implements FateStore { +public class ZooStore implements FateStore { - private static final Logger log = LoggerFactory.getLogger(ZooFatesStore.class); + private static final Logger log = LoggerFactory.getLogger(ZooStore.class); private String path; private ZooReaderWriter zk; @@ -147,7 +147,7 @@ private long parseTid(String txdir) { return Long.parseLong(txdir.split("_")[1], 16); } - public ZooFatesStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID) + public ZooStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID) throws KeeperException, InterruptedException { this.path = path; @@ -161,7 +161,7 @@ public ZooFatesStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID) /** * For testing only */ - ZooFatesStore() { + ZooStore() { lockID = null; } 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 0664f7eafef..dcf3a4a5c62 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 @@ -56,7 +56,7 @@ import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.FateTxId; import org.apache.accumulo.core.fate.ReadOnlyFateStore; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; @@ -761,7 +761,7 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); - ZooFatesStore zs = new ZooFatesStore<>(fateZkPath, zk, null); // TODO + ZooStore zs = new ZooStore<>(fateZkPath, zk, null); // TODO if (fateOpsCommand.cancel) { cancelSubmittedFateTxs(context, fateOpsCommand.txList); 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 7da6a92537c..0ded3f26fc6 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 @@ -71,7 +71,7 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.Fate; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.lock.ServiceLock; @@ -1082,7 +1082,7 @@ boolean canSuspendTablets() { } try { - final ZooFatesStore store = new ZooFatesStore<>(getZooKeeperRoot() + Constants.ZFATE, + final ZooStore store = new ZooStore<>(getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter(), managerLock.getLockID()); Fate f = new Fate<>(this, store, TraceRepo::toLogString, this::getParitionData, 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 11ce23147d2..ae46d81ed72 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 @@ -26,7 +26,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.ReadOnlyFateStore; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.metrics.MetricsUtil; import org.apache.accumulo.core.util.threads.ThreadPools; @@ -71,7 +71,7 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - this.zooStore = new ZooFatesStore<>(fateRootPath, context.getZooReaderWriter(), null); // TODO + this.zooStore = new ZooStore<>(fateRootPath, context.getZooReaderWriter(), null); // TODO } 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 596a9610ee7..29251cb19e2 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 @@ -33,7 +33,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.fate.ReadOnlyFateStore; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.volume.Volume; @@ -268,7 +268,7 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - final ReadOnlyFateStore fate = new ZooFatesStore<>( + final ReadOnlyFateStore fate = new ZooStore<>( context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter(), null); // TODO // maybe // have diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index b3adec7e468..0b3545285cf 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -48,7 +48,7 @@ import org.apache.accumulo.core.fate.FateTxId; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.manager.PartitionData; @@ -153,8 +153,8 @@ private ZooUtil.LockID createLockID() { @Timeout(30) public void testTransactionStatus() throws Exception { - final ZooFatesStore store = - new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); + final ZooStore store = + new ZooStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -213,8 +213,8 @@ public void testTransactionStatus() throws Exception { @Test public void testCancelWhileNew() throws Exception { - final ZooFatesStore store = - new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); + final ZooStore store = + new ZooStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -254,8 +254,8 @@ public void testCancelWhileNew() throws Exception { @Test public void testCancelWhileSubmittedAndRunning() throws Exception { - final ZooFatesStore store = - new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); + final ZooStore store = + new ZooStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); @@ -294,8 +294,8 @@ public void testCancelWhileSubmittedAndRunning() throws Exception { @Test public void testCancelWhileInCall() throws Exception { - final ZooFatesStore store = - new ZooFatesStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); + final ZooStore store = + new ZooStore(ZK_ROOT + Constants.ZFATE, zk, createLockID()); Manager manager = createMock(Manager.class); ServerContext sctx = createMock(ServerContext.class); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java index c9c75277977..cbc6c823907 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/MultipleFateInstancesIT.java @@ -33,7 +33,7 @@ import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.Repo; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.manager.PartitionData; @@ -74,41 +74,41 @@ public static void teardown() throws Exception { public void testZooStores() throws Exception { ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); - final ZooFatesStore zooFatesStore1 = new ZooFatesStore(FATE_DIR, zk, lock1); - final ZooFatesStore zooFatesStore2 = new ZooFatesStore(FATE_DIR, zk, lock2); + final ZooStore zooStore1 = new ZooStore(FATE_DIR, zk, lock1); + final ZooStore zooStore2 = new ZooStore(FATE_DIR, zk, lock2); Set allIds = new HashSet<>(); for (int i = 0; i < 100; i++) { - assertTrue(allIds.add(zooFatesStore1.create())); - assertTrue(allIds.add(zooFatesStore2.create())); + assertTrue(allIds.add(zooStore1.create())); + assertTrue(allIds.add(zooStore2.create())); } var pd1 = new PartitionData(0, 2); var pd2 = new PartitionData(1, 2); // nothing should be ready to run - assertFalse(zooFatesStore1.runnable(pd1).hasNext()); - assertFalse(zooFatesStore2.runnable(pd2).hasNext()); + assertFalse(zooStore1.runnable(pd1).hasNext()); + assertFalse(zooStore2.runnable(pd2).hasNext()); for (var txid : allIds) { if (txid % 2 == 0) { - var rfo = zooFatesStore1.reserve(txid); - assertTrue(zooFatesStore2.tryReserve(txid).isEmpty()); + var rfo = zooStore1.reserve(txid); + assertTrue(zooStore2.tryReserve(txid).isEmpty()); rfo.setStatus(ReadOnlyFateStore.TStatus.SUBMITTED); rfo.unreserve(0); } else { - var rfo = zooFatesStore2.reserve(txid); - assertTrue(zooFatesStore1.tryReserve(txid).isEmpty()); + var rfo = zooStore2.reserve(txid); + assertTrue(zooStore1.tryReserve(txid).isEmpty()); rfo.setStatus(ReadOnlyFateStore.TStatus.SUBMITTED); rfo.unreserve(0); } } HashSet runnable1 = new HashSet<>(); - zooFatesStore1.runnable(pd1).forEachRemaining(txid -> assertTrue(runnable1.add(txid))); + zooStore1.runnable(pd1).forEachRemaining(txid -> assertTrue(runnable1.add(txid))); HashSet runnable2 = new HashSet<>(); - zooFatesStore2.runnable(pd2).forEachRemaining(txid -> assertTrue(runnable2.add(txid))); + zooStore2.runnable(pd2).forEachRemaining(txid -> assertTrue(runnable2.add(txid))); assertFalse(runnable1.isEmpty()); assertFalse(runnable2.isEmpty()); @@ -116,8 +116,8 @@ public void testZooStores() throws Exception { assertEquals(allIds, Sets.union(runnable1, runnable2)); for (var txid : allIds) { - var rfo = zooFatesStore1.reserve(txid); - assertTrue(zooFatesStore2.tryReserve(txid).isEmpty()); + var rfo = zooStore1.reserve(txid); + assertTrue(zooStore2.tryReserve(txid).isEmpty()); rfo.delete(); rfo.unreserve(0); } @@ -163,8 +163,8 @@ public String getReturn() { public void testMultipleFateInstances() throws Exception { ZooUtil.LockID lock1 = new ZooUtil.LockID("/locks", "L1", 50); ZooUtil.LockID lock2 = new ZooUtil.LockID("/locks", "L2", 52); - final ZooFatesStore zooFatesStore1 = new ZooFatesStore<>(FATE_DIR, zk, lock1); - final ZooFatesStore zooFatesStore2 = new ZooFatesStore<>(FATE_DIR, zk, lock2); + final ZooStore zooStore1 = new ZooStore<>(FATE_DIR, zk, lock1); + final ZooStore zooStore2 = new ZooStore<>(FATE_DIR, zk, lock2); TestEnv testEnv1 = new TestEnv(); TestEnv testEnv2 = new TestEnv(); @@ -172,9 +172,9 @@ public void testMultipleFateInstances() throws Exception { AtomicReference partitionData1 = new AtomicReference<>(new PartitionData(0, 2)); AtomicReference partitionData2 = new AtomicReference<>(new PartitionData(1, 2)); - Fate fate1 = new Fate<>(testEnv1, zooFatesStore1, r -> "", partitionData1::get, + Fate fate1 = new Fate<>(testEnv1, zooStore1, r -> "", partitionData1::get, DefaultConfiguration.getInstance()); - Fate fate2 = new Fate<>(testEnv2, zooFatesStore2, r -> "", partitionData2::get, + Fate fate2 = new Fate<>(testEnv2, zooStore2, r -> "", partitionData2::get, DefaultConfiguration.getInstance()); Set submittedIds = new HashSet<>(); @@ -204,10 +204,10 @@ public void testMultipleFateInstances() throws Exception { // create a third fate instance ZooUtil.LockID lock3 = new ZooUtil.LockID("/locks", "L3", 54); - final ZooFatesStore zooFatesStore3 = new ZooFatesStore<>(FATE_DIR, zk, lock3); + final ZooStore zooStore3 = new ZooStore<>(FATE_DIR, zk, lock3); TestEnv testEnv3 = new TestEnv(); AtomicReference partitionData3 = new AtomicReference<>(new PartitionData(2, 3)); - Fate fate3 = new Fate<>(testEnv3, zooFatesStore3, r -> "", partitionData3::get, + Fate fate3 = new Fate<>(testEnv3, zooStore3, r -> "", partitionData3::get, DefaultConfiguration.getInstance()); // adjust the partition data for the existing fate instances, they should react to this change 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 984fa796547..eae14b32675 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 @@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; @@ -254,8 +254,8 @@ public void getFateStatus() { InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - ZooFatesStore zs = - new ZooFatesStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createLockID()); + ZooStore zs = + new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); @@ -345,8 +345,8 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); - ZooFatesStore zs = - new ZooFatesStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createLockID()); + ZooStore zs = + new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createLockID()); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); AdminUtil.FateStatus fateStatus = admin.getStatus(zs, zk, lockPath, null, null); 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 ad606b87b11..df09d626543 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 @@ -58,7 +58,7 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.AdminUtil.FateStatus; -import org.apache.accumulo.core.fate.ZooFatesStore; +import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.metadata.MetadataTable; @@ -226,8 +226,7 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); ZooReaderWriter zk = context.getZooReaderWriter(); - ZooFatesStore zs = - new ZooFatesStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, null); // TODO + ZooStore zs = new ZooStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, null); // TODO var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); return admin.getStatus(zs, zk, lockPath, null, null); } catch (KeeperException | InterruptedException e) { From 1aa878dd7d4412abd926394e221b5059fb92820a Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 14:29:33 -0500 Subject: [PATCH 09/12] use static import --- .../apache/accumulo/core/fate/AdminUtil.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index bd0dc8cbbf9..e7023084f34 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -33,6 +33,7 @@ import java.util.Map.Entry; import java.util.Set; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.zookeeper.FateLock; import org.apache.accumulo.core.fate.zookeeper.FateLock.FateLockPath; import org.apache.accumulo.core.fate.zookeeper.ZooReader; @@ -70,15 +71,15 @@ public AdminUtil(boolean exitOnError) { public static class TransactionStatus { private final long txid; - private final ReadOnlyFateStore.TStatus status; + private final TStatus status; private final String txName; private final List hlocks; private final List wlocks; private final String top; private final long timeCreated; - private TransactionStatus(Long tid, ReadOnlyFateStore.TStatus status, String txName, - List hlocks, List wlocks, String top, Long timeCreated) { + private TransactionStatus(Long tid, TStatus status, String txName, List hlocks, + List wlocks, String top, Long timeCreated) { this.txid = tid; this.status = status; @@ -98,7 +99,7 @@ public String getTxid() { return FastFormat.toHexString(txid); } - public ReadOnlyFateStore.TStatus getStatus() { + public TStatus getStatus() { return status; } @@ -217,7 +218,7 @@ public Map> getDanglingWaitingLocks() { * @return list of FATE transactions that match filter criteria */ public List getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, - EnumSet filterStatus) { + EnumSet filterStatus) { FateStatus status = getTransactionStatus(zs, filterTxid, filterStatus, Collections.>emptyMap(), Collections.>emptyMap()); @@ -239,8 +240,7 @@ public List getTransactionStatus(ReadOnlyFateStore zs, Set * @throws InterruptedException if process is interrupted. */ public FateStatus getStatus(ReadOnlyFateStore zs, ZooReader zk, - ServiceLock.ServiceLockPath lockPath, Set filterTxid, - EnumSet filterStatus) + ServiceLock.ServiceLockPath lockPath, Set filterTxid, EnumSet filterStatus) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); Map> waitingLocks = new HashMap<>(); @@ -333,7 +333,7 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, * @return current fate and lock status */ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, - EnumSet filterStatus, Map> heldLocks, + EnumSet filterStatus, Map> heldLocks, Map> waitingLocks) { List transactions = zs.list(); @@ -363,7 +363,7 @@ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filte top = repo.getName(); } - ReadOnlyFateStore.TStatus status = opStore.getStatus(); + TStatus status = opStore.getStatus(); long timeCreated = opStore.timeCreated(); @@ -376,8 +376,7 @@ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filte } - private boolean includeByStatus(ReadOnlyFateStore.TStatus status, - EnumSet filterStatus) { + private boolean includeByStatus(TStatus status, EnumSet filterStatus) { return (filterStatus == null) || filterStatus.contains(status); } @@ -392,8 +391,7 @@ public void printAll(ReadOnlyFateStore zs, ZooReader zk, public void print(ReadOnlyFateStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set filterTxid, - EnumSet filterStatus) - throws KeeperException, InterruptedException { + EnumSet filterStatus) throws KeeperException, InterruptedException { FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus); for (TransactionStatus txStatus : fateStatus.getTransactions()) { @@ -435,7 +433,7 @@ public boolean prepDelete(FateStore zs, ZooReaderWriter zk, ServiceLockPath p boolean state = false; var opStore = zs.reserve(txid); - ReadOnlyFateStore.TStatus ts = opStore.getStatus(); + TStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -472,7 +470,7 @@ public boolean prepFail(FateStore zs, ZooReaderWriter zk, ServiceLockPath zLo } boolean state = false; var opStore = zs.reserve(txid); - ReadOnlyFateStore.TStatus ts = opStore.getStatus(); + TStatus ts = opStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -482,7 +480,7 @@ public boolean prepFail(FateStore zs, ZooReaderWriter zk, ServiceLockPath zLo case IN_PROGRESS: case NEW: System.out.printf("Failing transaction: %016x (%s)%n", txid, ts); - opStore.setStatus(ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS); + opStore.setStatus(TStatus.FAILED_IN_PROGRESS); state = true; break; From b59496dee8e9caf1b4c8a56d6753327a5701d497 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 14:33:10 -0500 Subject: [PATCH 10/12] rename variable --- .../apache/accumulo/core/fate/AdminUtil.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index e7023084f34..4b765681dfe 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -341,9 +341,9 @@ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filte for (Long tid : transactions) { - var opStore = zs.read(tid); + var txStore = zs.read(tid); - String txName = (String) opStore.getTransactionInfo(Fate.TxInfo.TX_NAME); + String txName = (String) txStore.getTransactionInfo(Fate.TxInfo.TX_NAME); List hlocks = heldLocks.remove(tid); @@ -358,14 +358,14 @@ private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filte } String top = null; - ReadOnlyRepo repo = opStore.top(); + ReadOnlyRepo repo = txStore.top(); if (repo != null) { top = repo.getName(); } - TStatus status = opStore.getStatus(); + TStatus status = txStore.getStatus(); - long timeCreated = opStore.timeCreated(); + long timeCreated = txStore.timeCreated(); if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) { statuses.add(new TransactionStatus(tid, status, txName, hlocks, wlocks, top, timeCreated)); @@ -432,8 +432,8 @@ public boolean prepDelete(FateStore zs, ZooReaderWriter zk, ServiceLockPath p } boolean state = false; - var opStore = zs.reserve(txid); - TStatus ts = opStore.getStatus(); + var txStore = zs.reserve(txid); + TStatus ts = txStore.getStatus(); switch (ts) { case UNKNOWN: System.out.printf("Invalid transaction ID: %016x%n", txid); @@ -446,12 +446,12 @@ public boolean prepDelete(FateStore zs, ZooReaderWriter zk, ServiceLockPath p case FAILED_IN_PROGRESS: case SUCCESSFUL: System.out.printf("Deleting transaction: %016x (%s)%n", txid, ts); - opStore.delete(); + txStore.delete(); state = true; break; } - opStore.unreserve(0); + txStore.unreserve(0); return state; } From 019b7c3ceea259fd2b50143b8eb32a0275cb9b65 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 20:00:04 -0500 Subject: [PATCH 11/12] rename variable --- .../org/apache/accumulo/core/fate/Fate.java | 35 +++++++++---------- 1 file changed, 17 insertions(+), 18 deletions(-) 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 0848c020a00..28c6fe40e15 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 @@ -202,8 +202,7 @@ private class TransactionRunner implements Runnable { public void run() { while (keepRunning.get()) { long deferTime = 0; - // MERGE TODO rename to txStore - FateStore.FateTxStore opStore = null; + FateStore.FateTxStore txStore = null; try { var unreservedTid = workQueue.poll(100, MILLISECONDS); if (unreservedTid == null) { @@ -212,18 +211,18 @@ public void run() { queuedWork.remove(unreservedTid); var optionalopStore = store.tryReserve(unreservedTid); if (optionalopStore.isPresent()) { - opStore = optionalopStore.orElseThrow(); + txStore = optionalopStore.orElseThrow(); } else { continue; } - TStatus status = opStore.getStatus(); - Repo op = opStore.top(); + TStatus status = txStore.getStatus(); + Repo op = txStore.top(); if (status == FAILED_IN_PROGRESS) { - processFailed(opStore, op); + processFailed(txStore, op); } else if (status == SUBMITTED || status == IN_PROGRESS) { Repo prevOp = null; try { - deferTime = op.isReady(opStore.getID(), environment); + deferTime = op.isReady(txStore.getID(), environment); // Here, deferTime is only used to determine success (zero) or failure (non-zero), // proceeding on success and returning to the while loop on failure. @@ -231,16 +230,16 @@ public void run() { if (deferTime == 0) { prevOp = op; if (status == SUBMITTED) { - opStore.setStatus(IN_PROGRESS); + txStore.setStatus(IN_PROGRESS); } - op = op.call(opStore.getID(), environment); + op = op.call(txStore.getID(), environment); } else { continue; } } catch (Exception e) { - blockIfHadoopShutdown(opStore.getID(), e); - transitionToFailed(opStore, e); + blockIfHadoopShutdown(txStore.getID(), e); + transitionToFailed(txStore, e); continue; } @@ -248,18 +247,18 @@ public void run() { // transaction is finished String ret = prevOp.getReturn(); if (ret != null) { - opStore.setTransactionInfo(TxInfo.RETURN_VALUE, ret); + txStore.setTransactionInfo(TxInfo.RETURN_VALUE, ret); } - opStore.setStatus(SUCCESSFUL); - doCleanUp(opStore); + txStore.setStatus(SUCCESSFUL); + doCleanUp(txStore); } else { try { - opStore.push(op); + txStore.push(op); } catch (StackOverflowException e) { // the op that failed to push onto the stack was never executed, so no need to undo // it // just transition to failed and undo the ops that executed - transitionToFailed(opStore, e); + transitionToFailed(txStore, e); continue; } } @@ -267,8 +266,8 @@ public void run() { } catch (Exception e) { runnerLog.error("Uncaught exception in FATE runner thread.", e); } finally { - if (opStore != null) { - opStore.unreserve(deferTime); + if (txStore != null) { + txStore.unreserve(deferTime); } } } From 7f92e888e869b6e6057dd6f3fe3f3c0eb3abac05 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Wed, 6 Dec 2023 21:08:04 -0500 Subject: [PATCH 12/12] minimize diffs --- .../apache/accumulo/core/fate/ZooStore.java | 313 +++++++++--------- 1 file changed, 153 insertions(+), 160 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java index 256d2d48918..c8d84eae8a6 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java @@ -243,18 +243,71 @@ public Optional> tryReserve(long tid) { } } - private class ReadOnlyFateStoreImpl implements ReadOnlyFateTxStore { - private static final int RETRIES = 10; + private class FateTxStoreImpl implements FateTxStore { + + private final long tid; + private UUID reservationUUID; + private boolean deleted = false; - protected final long tid; + private FateTxStoreImpl(long tid) { + this.tid = tid; + this.reservationUUID = null; + } - protected ReadOnlyFateStoreImpl(long tid) { + private FateTxStoreImpl(long tid, UUID uuid) { this.tid = tid; + this.reservationUUID = Objects.requireNonNull(uuid); + } + + private void unreserve() { + Preconditions.checkState(reservationUUID != null); + try { + if (!deleted) { + zk.mutateExisting(getTXPath(tid), currentValue -> { + var nodeVal = new NodeValue(currentValue); + if (nodeVal.uuid.equals(reservationUUID.toString())) { + return new NodeValue(nodeVal.status, "", "").serialize(); + } else { + // possible this is running a 2nd time in zk server fault conditions and its first + // write went through + return null; + } + }); + } + reservationUUID = null; + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } } + @Override + public void unreserve(long deferTime) { + + if (deferTime < 0) { + throw new IllegalArgumentException("deferTime < 0 : " + deferTime); + } + + if (deferTime > 0) { + // add to defered before actually unreserving + defered.put(tid, System.currentTimeMillis() + deferTime); + } + + unreserve(); + } + + private void verifyReserved(boolean isWrite) { + if (reservationUUID == null && isWrite) { + throw new IllegalStateException( + "Attempted write on unreserved FATE transaction." + FateTxId.formatTid(getID())); + } + } + + private static final int RETRIES = 10; + @Override public Repo top() { - checkState(false); + verifyReserved(false); + for (int i = 0; i < RETRIES; i++) { String txpath = getTXPath(tid); try { @@ -283,8 +336,8 @@ public Repo top() { return null; } - String findTop(String txpath) throws KeeperException, InterruptedException { - checkState(false); + private String findTop(String txpath) throws KeeperException, InterruptedException { + verifyReserved(false); List ops = zk.getChildren(txpath); ops = new ArrayList<>(ops); @@ -304,8 +357,43 @@ String findTop(String txpath) throws KeeperException, InterruptedException { return max; } + @Override + public void push(Repo repo) throws StackOverflowException { + verifyReserved(true); + + String txpath = getTXPath(tid); + try { + String top = findTop(txpath); + if (top != null && Long.parseLong(top.split("_")[1]) > 100) { + throw new StackOverflowException("Repo stack size too large"); + } + + zk.putPersistentSequential(txpath + "/repo_", serialize(repo)); + } catch (StackOverflowException soe) { + throw soe; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void pop() { + verifyReserved(true); + + try { + String txpath = getTXPath(tid); + String top = findTop(txpath); + if (top == null) { + throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid)); + } + zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + private TStatus _getStatus() { - checkState(false); + verifyReserved(false); try { return new NodeValue(zk.getData(getTXPath(tid))).status; } catch (NoNodeException nne) { @@ -317,13 +405,13 @@ private TStatus _getStatus() { @Override public TStatus getStatus() { - checkState(false); + verifyReserved(false); return _getStatus(); } @Override public TStatus waitForStatusChange(EnumSet expected) { - checkState(false); + verifyReserved(false); // 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 @@ -351,9 +439,58 @@ public TStatus waitForStatusChange(EnumSet expected) { } } + @Override + public void setStatus(TStatus status) { + verifyReserved(true); + + try { + zk.mutateExisting(getTXPath(tid), currentValue -> { + var nodeVal = new NodeValue(currentValue); + Preconditions.checkState(nodeVal.uuid.equals(reservationUUID.toString()), + "Tried to set status for %s and it was not reserved", FateTxId.formatTid(tid)); + return new NodeValue(status, nodeVal.lock, nodeVal.uuid).serialize(); + }); + } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void delete() { + verifyReserved(true); + try { + zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP); + deleted = true; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + public void setTransactionInfo(Fate.TxInfo txInfo, Serializable so) { + verifyReserved(true); + + try { + if (so instanceof String) { + zk.putPersistentData(getTXPath(tid) + "/" + txInfo, ("S " + so).getBytes(UTF_8), + NodeExistsPolicy.OVERWRITE); + } else { + byte[] sera = serialize(so); + byte[] data = new byte[sera.length + 2]; + System.arraycopy(sera, 0, data, 2, sera.length); + data[0] = 'O'; + data[1] = ' '; + zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE); + } + } catch (KeeperException | InterruptedException e2) { + throw new IllegalStateException(e2); + } + } + @Override public Serializable getTransactionInfo(Fate.TxInfo txInfo) { - checkState(false); + verifyReserved(false); + try { byte[] data = zk.getData(getTXPath(tid) + "/" + txInfo); @@ -375,7 +512,8 @@ public Serializable getTransactionInfo(Fate.TxInfo txInfo) { @Override public long timeCreated() { - checkState(false); + verifyReserved(false); + try { Stat stat = zk.getZooKeeper().exists(getTXPath(tid), false); return stat.getCtime(); @@ -386,13 +524,13 @@ public long timeCreated() { @Override public long getID() { - checkState(false); + verifyReserved(false); return tid; } @Override public List> getStack() { - checkState(false); + verifyReserved(false); String txpath = getTXPath(tid); outer: while (true) { @@ -430,156 +568,11 @@ public List> getStack() { return dops; } } - - protected void checkState(boolean unreserving) {} - - } - - private class FateTxStoreImpl extends ReadOnlyFateStoreImpl implements FateTxStore { - - private boolean reserved = true; - private boolean deleted = false; - - private final UUID uuid; - - protected FateTxStoreImpl(long tid, UUID uuid) { - super(tid); - this.uuid = Objects.requireNonNull(uuid); - } - - @Override - public void push(Repo repo) throws StackOverflowException { - checkState(false); - String txpath = getTXPath(tid); - try { - String top = findTop(txpath); - if (top != null && Long.parseLong(top.split("_")[1]) > 100) { - throw new StackOverflowException("Repo stack size too large"); - } - - zk.putPersistentSequential(txpath + "/repo_", serialize(repo)); - } catch (StackOverflowException soe) { - throw soe; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void pop() { - checkState(false); - try { - String txpath = getTXPath(tid); - String top = findTop(txpath); - if (top == null) { - throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid)); - } - zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP); - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void setStatus(TStatus status) { - checkState(false); - try { - zk.mutateExisting(getTXPath(tid), currentValue -> { - var nodeVal = new NodeValue(currentValue); - Preconditions.checkState(nodeVal.uuid.equals(uuid.toString()), - "Tried to set status for %s and it was not reserved", FateTxId.formatTid(tid)); - return new NodeValue(status, nodeVal.lock, nodeVal.uuid).serialize(); - }); - } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void delete() { - checkState(false); - try { - zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP); - deleted = true; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void setTransactionInfo(Fate.TxInfo txInfo, Serializable so) { - checkState(false); - try { - if (so instanceof String) { - zk.putPersistentData(getTXPath(tid) + "/" + txInfo, ("S " + so).getBytes(UTF_8), - NodeExistsPolicy.OVERWRITE); - } else { - byte[] sera = serialize(so); - byte[] data = new byte[sera.length + 2]; - System.arraycopy(sera, 0, data, 2, sera.length); - data[0] = 'O'; - data[1] = ' '; - zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE); - } - } catch (KeeperException | InterruptedException e2) { - throw new IllegalStateException(e2); - } - } - - private void unreserve() { - checkState(true); - try { - if (!deleted) { - zk.mutateExisting(getTXPath(tid), currentValue -> { - var nodeVal = new NodeValue(currentValue); - if (nodeVal.uuid.equals(uuid.toString())) { - return new NodeValue(nodeVal.status, "", "").serialize(); - } else { - // possible this is running a 2nd time in zk server fault conditions and its first - // write went through - return null; - } - }); - } - reserved = false; - } catch (KeeperException | InterruptedException | AcceptableThriftTableOperationException e) { - throw new IllegalStateException(e); - } - } - - @Override - public void unreserve(long deferTime) { - - if (deferTime < 0) { - throw new IllegalArgumentException("deferTime < 0 : " + deferTime); - } - - if (deferTime > 0) { - // add to defered before actually unreserving - defered.put(tid, System.currentTimeMillis() + deferTime); - } - - unreserve(); - } - - @Override - protected void checkState(boolean unreserving) { - super.checkState(unreserving); - if (!reserved) { - throw new IllegalStateException("Attempted to use fate store " + FateTxId.formatTid(getID()) - + " after unreserving it."); - } - - if (!unreserving && deleted) { - throw new IllegalStateException("Attempted to use fate store for " - + FateTxId.formatTid(getID()) + " after deleting it."); - } - } } @Override public ReadOnlyFateTxStore read(long tid) { - return new ReadOnlyFateStoreImpl(tid); + return new FateTxStoreImpl(tid); } @Override