From fbecbd6ccc95cd927fe17af28aec9e5c6072533e Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 26 Jan 2024 09:17:50 -0500 Subject: [PATCH 01/16] Add support for creating a FATE transaction by key This updates FATE to support creating a transaction by a key which will be hashed to a transaction id. This can be used to guarantee only one transaction exists at the same time if the key is the same. Currently the key is just a byte array so anything can be used for the key. This closes #4183 --- .../accumulo/core/fate/AbstractFateStore.java | 91 ++++++++++---- .../org/apache/accumulo/core/fate/Fate.java | 2 +- .../apache/accumulo/core/fate/FateStore.java | 2 + .../accumulo/core/fate/ReadOnlyFateStore.java | 7 ++ .../core/fate/WrappedFateTxStore.java | 14 +++ .../apache/accumulo/core/fate/ZooStore.java | 47 ++++++-- .../core/fate/accumulo/AccumuloStore.java | 53 ++++++++ .../core/fate/accumulo/FateMutator.java | 2 + .../core/fate/accumulo/FateMutatorImpl.java | 9 ++ .../core/fate/accumulo/schema/FateSchema.java | 3 + .../accumulo/core/logging/FateLogger.java | 9 ++ .../apache/accumulo/core/fate/TestStore.java | 45 +++++-- .../org/apache/accumulo/test/fate/FateIT.java | 7 +- .../accumulo/test/fate/FateTestRunner.java | 15 ++- .../test/fate/accumulo/AccumuloFateIT.java | 2 +- .../fate/accumulo/AccumuloStoreFateIT.java | 3 +- .../test/fate/accumulo/FateStoreIT.java | 114 +++++++++++++++++- .../test/fate/zookeeper/ZooStoreFateIT.java | 2 +- .../test/fate/zookeeper/ZookeeperFateIT.java | 2 +- 19 files changed, 373 insertions(+), 56 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 51670aacfda..9abfa5ebb87 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -27,6 +27,7 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.io.UncheckedIOException; +import java.util.Arrays; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -40,10 +41,13 @@ import java.util.stream.Stream; import org.apache.accumulo.core.fate.Fate.TxInfo; +import org.apache.accumulo.core.util.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hashing; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -243,10 +247,43 @@ public int getDeferredCount() { } } + @Override + public long create(byte[] key) { + HashCode hashCode = Hashing.murmur3_128().hashBytes(key); + long tid = hashCode.asLong() & 0x7fffffffffffffffL; + + Pair> statusAndKey = getStatusAndKey(tid); + TStatus status = statusAndKey.getFirst(); + Optional tKey = statusAndKey.getSecond(); + + // Case 1: Status of UNKNOWN means doesn't exist, so we can create + if (status == TStatus.UNKNOWN) { + create(tid, key); + // Case 2: Status is NEW so this is unseeded, we can return and allow the calling code + // to reserve/seed as long as the existing key is the same and not different as that would + // mean a collision + } else if (status == TStatus.NEW) { + Preconditions.checkState(tKey.isPresent(), "Tx key column is missing"); + Preconditions.checkState(Arrays.equals(key, tKey.orElseThrow()), + "Collision detected for tid %s", tid); + // Case 3: Status is some other state which means already in progress + } else { + throw new IllegalStateException("Existing transaction already exists for: " + tid); + } + + return tid; + } + + protected abstract void create(long tid, byte[] key); + + protected abstract Pair> getStatusAndKey(long tid); + protected abstract Stream getTransactions(); protected abstract TStatus _getStatus(long tid); + protected abstract Optional getKey(long tid); + protected abstract FateTxStore newFateTxStore(long tid, boolean isReserved); protected abstract class AbstractFateTxStoreImpl implements FateTxStore { @@ -343,34 +380,46 @@ public TStatus getStatus() { return status; } + @Override + public Optional getKey() { + verifyReserved(false); + return AbstractFateStore.this.getKey(tid); + } + + @Override + public Pair> getStatusAndKey() { + verifyReserved(false); + return AbstractFateStore.this.getStatusAndKey(tid); + } + @Override public long getID() { return tid; } + } - protected byte[] serializeTxInfo(Serializable so) { - if (so instanceof String) { - return ("S " + so).getBytes(UTF_8); - } 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] = ' '; - return data; - } + protected byte[] serializeTxInfo(Serializable so) { + if (so instanceof String) { + return ("S " + so).getBytes(UTF_8); + } 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] = ' '; + return data; } + } - protected Serializable deserializeTxInfo(TxInfo txInfo, byte[] data) { - 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); - } + protected Serializable deserializeTxInfo(TxInfo txInfo, byte[] data) { + 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); } } } 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 55845078475..3147db48151 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 @@ -77,7 +77,7 @@ public class Fate { private final Thread workFinder; public enum TxInfo { - TX_NAME, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE + TX_NAME, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE, TX_KEY } /** 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 b5ccae52684..189b51d475f 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 @@ -39,6 +39,8 @@ public interface FateStore extends ReadOnlyFateStore { */ long create(); + long create(byte[] key); + /** * An interface that allows read/write access to the data related to a single fate operation. */ 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 4ddf9afae5c..9e9f746bb2a 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 @@ -21,10 +21,13 @@ import java.io.Serializable; import java.util.EnumSet; import java.util.List; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongConsumer; import java.util.stream.Stream; +import org.apache.accumulo.core.util.Pair; + /** * Read only access to a Transaction Store. * @@ -87,6 +90,10 @@ interface ReadOnlyFateTxStore { */ TStatus getStatus(); + Optional getKey(); + + Pair> getStatusAndKey(); + /** * Wait for the status of a transaction to change * diff --git a/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java b/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java index 1d8c7126c22..b18da9d68f9 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java @@ -21,8 +21,12 @@ import java.io.Serializable; import java.util.EnumSet; import java.util.List; +import java.util.Optional; import java.util.concurrent.TimeUnit; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; +import org.apache.accumulo.core.util.Pair; + public class WrappedFateTxStore implements FateStore.FateTxStore { protected final FateStore.FateTxStore wrapped; @@ -55,6 +59,16 @@ public FateStore.TStatus getStatus() { return wrapped.getStatus(); } + @Override + public Optional getKey() { + return wrapped.getKey(); + } + + @Override + public Pair> getStatusAndKey() { + return wrapped.getStatusAndKey(); + } + @Override public void setStatus(FateStore.TStatus status) { wrapped.setStatus(status); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java index fb8f7ee7ed4..2d07ac71dee 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 @@ -27,13 +27,16 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.function.Supplier; import java.util.stream.Stream; +import org.apache.accumulo.core.fate.Fate.TxInfo; 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.accumulo.core.util.Pair; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; @@ -91,6 +94,27 @@ public long create() { } } + @Override + protected void create(long tid, byte[] key) { + // TODO: Should we somehow make this Atomic or clean up on failure to make sure + // that either both of these writes happen or none happen? + try { + zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8), + NodeExistsPolicy.FAIL); + // The key was already used to generate the tid but we still need to store it + // separate to check later for collision detection + zk.putPersistentData(getTXPath(tid) + "/" + TxInfo.TX_KEY, serializeTxInfo(key), + NodeExistsPolicy.OVERWRITE); + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + @Override + protected Pair> getStatusAndKey(long tid) { + return new Pair<>(_getStatus(tid), getKey(tid)); + } + private class FateTxStoreImpl extends AbstractFateTxStoreImpl { private FateTxStoreImpl(long tid, boolean isReserved) { @@ -227,13 +251,7 @@ public void setTransactionInfo(Fate.TxInfo txInfo, Serializable so) { public Serializable getTransactionInfo(Fate.TxInfo txInfo) { verifyReserved(false); - try { - return deserializeTxInfo(txInfo, zk.getData(getTXPath(tid) + "/" + txInfo)); - } catch (NoNodeException nne) { - return null; - } catch (KeeperException | InterruptedException e) { - throw new IllegalStateException(e); - } + return ZooStore.this.getTransactionInfo(txInfo, tid); } @Override @@ -290,6 +308,16 @@ public List> getStack() { } } + private Serializable getTransactionInfo(TxInfo txInfo, long tid) { + try { + return deserializeTxInfo(txInfo, zk.getData(getTXPath(tid) + "/" + txInfo)); + } catch (NoNodeException nne) { + return null; + } catch (KeeperException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + @Override protected TStatus _getStatus(long tid) { try { @@ -301,6 +329,11 @@ protected TStatus _getStatus(long tid) { } } + @Override + protected Optional getKey(long tid) { + return Optional.ofNullable((byte[]) getTransactionInfo(TxInfo.TX_KEY, tid)); + } + @Override protected FateTxStore newFateTxStore(long tid, boolean isReserved) { return new FateTxStoreImpl(tid, isReserved); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index b5dc999d425..845ae60e885 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -22,6 +22,7 @@ import java.io.Serializable; import java.util.List; +import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.function.Function; @@ -31,9 +32,12 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.Fate.TxInfo; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; @@ -44,6 +48,7 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.ColumnFQ; import org.apache.accumulo.core.util.FastFormat; +import org.apache.accumulo.core.util.Pair; import org.apache.hadoop.io.Text; import com.google.common.base.Preconditions; @@ -80,6 +85,13 @@ public long create() { return tid; } + @Override + protected void create(long tid, byte[] key) { + // TODO: conditional mutation should be used to verify tid is new + newMutator(tid).putStatus(TStatus.NEW).putKey(key).putCreateTime(System.currentTimeMillis()) + .mutate(); + } + @Override protected Stream getTransactions() { try { @@ -109,6 +121,44 @@ protected TStatus _getStatus(long tid) { }); } + @Override + protected Optional getKey(long tid) { + return scanTx(scanner -> { + scanner.setRange(getRow(tid)); + TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); + return scanner.stream().map(e -> e.getValue().get()).findFirst(); + }); + } + + @Override + protected Pair> getStatusAndKey(long tid) { + return scanTx(scanner -> { + scanner.setRange(getRow(tid)); + TxColumnFamily.STATUS_COLUMN.fetch(scanner); + TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); + + TStatus status = null; + byte[] key = null; + + for (Entry entry : scanner) { + final String qual = entry.getKey().getColumnQualifierData().toString(); + switch (qual) { + case TxColumnFamily.STATUS: + status = TStatus.valueOf(entry.getValue().toString()); + break; + case TxInfoColumnFamily.TX_KEY: + key = entry.getValue().get(); + break; + default: + throw new IllegalStateException("Unexpected column qualifier: " + qual); + } + } + + return new Pair<>(Optional.ofNullable(status).orElse(TStatus.UNKNOWN), + Optional.ofNullable(key)); + }); + } + @Override protected FateTxStore newFateTxStore(long tid, boolean isReserved) { return new FateTxStoreImpl(tid, isReserved); @@ -191,6 +241,9 @@ public Serializable getTransactionInfo(TxInfo txInfo) { case TX_AGEOFF: cq = TxInfoColumnFamily.TX_AGEOFF_COLUMN; break; + case TX_KEY: + cq = TxInfoColumnFamily.TX_KEY_COLUMN; + break; default: throw new IllegalArgumentException("Unexpected TxInfo type " + txInfo); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java index 4caf5985bd9..f69521852bb 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java @@ -38,6 +38,8 @@ public interface FateMutator { FateMutator putAgeOff(byte[] data); + FateMutator putKey(byte[] data); + FateMutator putTxInfo(Fate.TxInfo txInfo, byte[] data); FateMutator putRepo(int position, Repo repo); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java index 90d22008d59..612d9f2fd2f 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java @@ -96,6 +96,12 @@ public FateMutator putAgeOff(byte[] data) { return this; } + @Override + public FateMutator putKey(byte[] data) { + TxInfoColumnFamily.TX_KEY_COLUMN.put(mutation, new Value(data)); + return this; + } + @Override public FateMutator putTxInfo(TxInfo txInfo, byte[] data) { switch (txInfo) { @@ -114,6 +120,9 @@ public FateMutator putTxInfo(TxInfo txInfo, byte[] data) { case TX_AGEOFF: putAgeOff(data); break; + case TX_KEY: + putKey(data); + break; default: throw new IllegalArgumentException("Unexpected TxInfo type: " + txInfo); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java index 7e4e639a7ce..41205dd1f4a 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java @@ -41,6 +41,9 @@ public static class TxInfoColumnFamily { public static final String TX_NAME = "txname"; public static final ColumnFQ TX_NAME_COLUMN = new ColumnFQ(NAME, new Text(TX_NAME)); + public static final String TX_KEY = "txkey"; + public static final ColumnFQ TX_KEY_COLUMN = new ColumnFQ(NAME, new Text(TX_KEY)); + public static final String AUTO_CLEAN = "autoclean"; public static final ColumnFQ AUTO_CLEAN_COLUMN = new ColumnFQ(NAME, new Text(AUTO_CLEAN)); 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 189df12362c..2dbcf345616 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 @@ -142,6 +142,15 @@ public long create() { } return tid; } + + @Override + public long create(byte[] key) { + long tid = store.create(key); + if (storeLog.isTraceEnabled()) { + storeLog.trace("{} created fate transaction", formatTid(tid)); + } + return tid; + } }; } } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 5df2e0fa0a0..0f0ca3fc0e6 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -32,22 +32,37 @@ import java.util.function.LongConsumer; import java.util.stream.Stream; +import org.apache.accumulo.core.util.Pair; + +import com.google.common.hash.HashCode; +import com.google.common.hash.Hashing; + /** * Transient in memory store for transactions. */ public class TestStore implements FateStore { private long nextId = 1; - private Map statuses = new HashMap<>(); - private Map> txInfos = new HashMap<>(); - private Set reserved = new HashSet<>(); + private final Map>> statuses = new HashMap<>(); + private final Map> txInfos = new HashMap<>(); + private final Set reserved = new HashSet<>(); @Override public long create() { - statuses.put(nextId, TStatus.NEW); + statuses.put(nextId, new Pair<>(TStatus.NEW, Optional.empty())); return nextId++; } + @Override + public long create(byte[] key) { + HashCode hashCode = Hashing.murmur3_128().hashBytes(key); + long tid = hashCode.asLong() & 0x7fffffffffffffffL; + if (statuses.putIfAbsent(tid, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { + throw new IllegalStateException("Transaction with tid " + tid + " already exists"); + } + return tid; + } + @Override public FateTxStore reserve(long tid) { if (reserved.contains(tid)) { @@ -89,14 +104,25 @@ public List> getStack() { @Override public TStatus getStatus() { + return getStatusAndKey().getFirst(); + } + + @Override + public Optional getKey() { + return getStatusAndKey().getSecond(); + } + + @Override + public Pair> getStatusAndKey() { if (!reserved.contains(tid)) { throw new IllegalStateException(); } - TStatus status = statuses.get(tid); + Pair> status = statuses.get(tid); if (status == null) { - return TStatus.UNKNOWN; + return new Pair<>(TStatus.UNKNOWN, Optional.empty()); } + return status; } @@ -140,10 +166,11 @@ public void setStatus(TStatus status) { if (!reserved.contains(tid)) { throw new IllegalStateException(); } - if (!statuses.containsKey(tid)) { + Pair> currentStatus = statuses.get(tid); + if (currentStatus == null) { throw new IllegalStateException(); } - statuses.put(tid, status); + statuses.put(tid, new Pair<>(status, currentStatus.getSecond())); } @Override @@ -186,7 +213,7 @@ public long getTxid() { @Override public TStatus getStatus() { - return e.getValue(); + return e.getValue().getFirst(); } }); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java index 380876aee07..0981d201a3f 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java @@ -43,23 +43,20 @@ import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.test.fate.FateTestRunner.TestEnv; import org.apache.accumulo.test.util.Wait; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class FateIT extends SharedMiniClusterBase implements FateTestRunner { +public abstract class FateIT extends SharedMiniClusterBase implements FateTestRunner { private static final Logger LOG = LoggerFactory.getLogger(FateIT.class); private static CountDownLatch callStarted; private static CountDownLatch finishCall; - public static class TestEnv { - - } - public static class TestRepo implements Repo { private static final long serialVersionUID = 1L; diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java b/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java index b87702df910..15df2404dd9 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java @@ -20,18 +20,21 @@ import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.server.ServerContext; -import org.apache.accumulo.test.fate.FateIT.TestEnv; +import org.apache.accumulo.test.fate.FateTestRunner.TestEnv; -public interface FateTestRunner { +public interface FateTestRunner { - void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception; + void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception; - default void executeTest(FateTestExecutor testMethod) throws Exception { + default void executeTest(FateTestExecutor testMethod) throws Exception { executeTest(testMethod, 100_000); } - interface FateTestExecutor { - void execute(FateStore store, ServerContext sctx) throws Exception; + interface FateTestExecutor { + void execute(FateStore store, ServerContext sctx) throws Exception; } + class TestEnv { + + } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java index 0dec7e442b2..44966bd4881 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java @@ -51,7 +51,7 @@ public static void teardown() { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { table = getUniqueNames(1)[0]; try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java index 7ef3e575deb..2daee958180 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java @@ -38,7 +38,7 @@ public static void teardown() { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { String table = getUniqueNames(1)[0]; try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { @@ -47,4 +47,5 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exc getCluster().getServerContext()); } } + } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index 8ddd3b81b0e..a30c466d237 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -18,13 +18,20 @@ */ package org.apache.accumulo.test.fate.accumulo; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; import java.time.Duration; import java.util.HashSet; import java.util.List; @@ -34,6 +41,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.FateStore.FateTxStore; @@ -42,13 +51,14 @@ import org.apache.accumulo.core.fate.StackOverflowException; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.server.ServerContext; -import org.apache.accumulo.test.fate.FateIT.TestEnv; import org.apache.accumulo.test.fate.FateIT.TestRepo; import org.apache.accumulo.test.fate.FateTestRunner; +import org.apache.accumulo.test.fate.FateTestRunner.TestEnv; import org.apache.accumulo.test.util.Wait; +import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; -public abstract class FateStoreIT extends SharedMiniClusterBase implements FateTestRunner { +public abstract class FateStoreIT extends SharedMiniClusterBase implements FateTestRunner { @Override protected Duration defaultTimeout() { @@ -69,6 +79,7 @@ protected void testReadWrite(FateStore store, ServerContext sctx) long tid = store.create(); FateTxStore txStore = store.reserve(tid); assertTrue(txStore.timeCreated() > 0); + assertFalse(txStore.getKey().isPresent()); assertEquals(1, store.list().count()); // Push a test FATE op and verify we can read it back @@ -216,7 +227,104 @@ protected void testDeferredOverflow(FateStore store, ServerContext sctx } finally { executor.shutdownNow(); // Cleanup so we don't interfere with other tests - store.list().forEach(fateIdStatus -> store.reserve(fateIdStatus.getTxid()).delete()); + // All stores should already be unreserved + store.list() + .forEach(fateIdStatus -> store.tryReserve(fateIdStatus.getTxid()).orElseThrow().delete()); + } + } + + @Test + public void testCreateWithKey() throws Exception { + executeTest(this::testCreateWithKey); + } + + protected void testCreateWithKey(FateStore store, ServerContext sctx) { + KeyExtent ke1 = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); + KeyExtent ke2 = new KeyExtent(TableId.of("tableId2"), new Text("zzz"), new Text("aaa")); + + byte[] key1 = serialize(ke1); + long tid1 = store.create(key1); + + byte[] key2 = serialize(ke2); + long tid2 = store.create(key2); + assertNotEquals(tid1, tid2); + + FateTxStore txStore1 = store.reserve(tid1); + FateTxStore txStore2 = store.reserve(tid2); + try { + assertTrue(txStore1.timeCreated() > 0); + assertEquals(TStatus.NEW, txStore1.getStatus()); + assertArrayEquals(key1, txStore1.getKey().orElseThrow()); + + assertTrue(txStore2.timeCreated() > 0); + assertEquals(TStatus.NEW, txStore2.getStatus()); + assertArrayEquals(key2, txStore2.getKey().orElseThrow()); + + assertEquals(2, store.list().count()); + } finally { + txStore1.delete(); + txStore2.delete(); + } + } + + @Test + public void testCreateWithKeyDuplicate() throws Exception { + executeTest(this::testCreateWithKeyDuplicate); + } + + protected void testCreateWithKeyDuplicate(FateStore store, ServerContext sctx) { + KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); + + // Creating with the same key should be fine if the status is NEW + // It should just return the same id and allow us to continue reserving + byte[] key = serialize(ke); + long tid1 = store.create(key); + long tid2 = store.create(key); + assertEquals(tid1, tid2); + + FateTxStore txStore = store.reserve(tid1); + try { + assertTrue(txStore.timeCreated() > 0); + assertEquals(TStatus.NEW, txStore.getStatus()); + assertArrayEquals(key, txStore.getKey().orElseThrow()); + assertEquals(1, store.list().count()); + } finally { + txStore.delete(); + } + } + + @Test + public void testCreateWithKeyInProgress() throws Exception { + executeTest(this::testCreateWithKeyInProgress); + } + + protected void testCreateWithKeyInProgress(FateStore store, ServerContext sctx) { + KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); + + byte[] key = serialize(ke); + long tid1 = store.create(key); + + FateTxStore txStore = store.reserve(tid1); + try { + assertTrue(txStore.timeCreated() > 0); + txStore.setStatus(TStatus.IN_PROGRESS); + + // We have an existing transaction with the same key in progress + // so should not be allowed + assertThrows(IllegalStateException.class, () -> store.create(key)); + } finally { + txStore.delete(); + } + } + + private byte[] serialize(KeyExtent ke) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos)) { + ke.writeTo(dos); + dos.close(); + return baos.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java index 04530e317fa..bbb6aa1f6b5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java @@ -61,7 +61,7 @@ public static void teardown() throws Exception { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java index 64a18d38a1c..ed198f73c87 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java @@ -64,7 +64,7 @@ public static void teardown() throws Exception { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); From 1e6a471ecf2136bfd80335377ed9631633e3ba44 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 2 Feb 2024 08:36:13 -0500 Subject: [PATCH 02/16] Updates after the change from tid to FateId --- .../accumulo/core/fate/AbstractFateStore.java | 22 +++++++++------- .../apache/accumulo/core/fate/FateStore.java | 2 +- .../apache/accumulo/core/fate/ZooStore.java | 25 +++++++++++------- .../core/fate/accumulo/AccumuloStore.java | 21 +++++++++------ .../accumulo/core/logging/FateLogger.java | 8 +++--- .../apache/accumulo/core/fate/TestStore.java | 18 ++++++------- .../test/fate/accumulo/FateStoreIT.java | 26 +++++++++---------- 7 files changed, 67 insertions(+), 55 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 919eda5f5ea..1ea96d56e45 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -244,17 +244,18 @@ public int getDeferredCount() { } @Override - public long create(byte[] key) { + public FateId create(byte[] key) { HashCode hashCode = Hashing.murmur3_128().hashBytes(key); long tid = hashCode.asLong() & 0x7fffffffffffffffL; + FateId fateId = FateId.from(getInstanceType(), tid); - Pair> statusAndKey = getStatusAndKey(tid); + Pair> statusAndKey = getStatusAndKey(fateId); TStatus status = statusAndKey.getFirst(); Optional tKey = statusAndKey.getSecond(); // Case 1: Status of UNKNOWN means doesn't exist, so we can create if (status == TStatus.UNKNOWN) { - create(tid, key); + create(fateId, key); // Case 2: Status is NEW so this is unseeded, we can return and allow the calling code // to reserve/seed as long as the existing key is the same and not different as that would // mean a collision @@ -267,21 +268,22 @@ public long create(byte[] key) { throw new IllegalStateException("Existing transaction already exists for: " + tid); } - return tid; + return fateId; } - protected abstract void create(long tid, byte[] key); + protected abstract void create(FateId fateId, byte[] key); - protected abstract Pair> getStatusAndKey(long tid); + protected abstract Pair> getStatusAndKey(FateId fateId); protected abstract Stream getTransactions(); protected abstract TStatus _getStatus(FateId fateId); - protected abstract Optional getKey(long tid); - + protected abstract Optional getKey(FateId fateId); + protected abstract FateTxStore newFateTxStore(FateId fateId, boolean isReserved); + protected abstract FateInstanceType getInstanceType(); protected abstract class AbstractFateTxStoreImpl implements FateTxStore { protected final FateId fateId; @@ -378,13 +380,13 @@ public TStatus getStatus() { @Override public Optional getKey() { verifyReserved(false); - return AbstractFateStore.this.getKey(tid); + return AbstractFateStore.this.getKey(fateId); } @Override public Pair> getStatusAndKey() { verifyReserved(false); - return AbstractFateStore.this.getStatusAndKey(tid); + return AbstractFateStore.this.getStatusAndKey(fateId); } @Override 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 c338e05d125..6c91d8d11ab 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 @@ -39,7 +39,7 @@ public interface FateStore extends ReadOnlyFateStore { */ FateId create(); - long create(byte[] key); + FateId create(byte[] key); /** * An interface that allows read/write access to the data related to a single fate operation. 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 54ed9ef2a64..b48fb189cb4 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 @@ -96,15 +96,15 @@ public FateId create() { } @Override - protected void create(long tid, byte[] key) { + protected void create(FateId fateId, byte[] key) { // TODO: Should we somehow make this Atomic or clean up on failure to make sure // that either both of these writes happen or none happen? try { - zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8), + zk.putPersistentData(getTXPath(fateId), TStatus.NEW.name().getBytes(UTF_8), NodeExistsPolicy.FAIL); // The key was already used to generate the tid but we still need to store it // separate to check later for collision detection - zk.putPersistentData(getTXPath(tid) + "/" + TxInfo.TX_KEY, serializeTxInfo(key), + zk.putPersistentData(getTXPath(fateId) + "/" + TxInfo.TX_KEY, serializeTxInfo(key), NodeExistsPolicy.OVERWRITE); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); @@ -112,8 +112,8 @@ protected void create(long tid, byte[] key) { } @Override - protected Pair> getStatusAndKey(long tid) { - return new Pair<>(_getStatus(tid), getKey(tid)); + protected Pair> getStatusAndKey(FateId fateId) { + return new Pair<>(_getStatus(fateId), getKey(fateId)); } private class FateTxStoreImpl extends AbstractFateTxStoreImpl { @@ -252,7 +252,7 @@ public void setTransactionInfo(Fate.TxInfo txInfo, Serializable so) { public Serializable getTransactionInfo(Fate.TxInfo txInfo) { verifyReserved(false); - return deserializeTxInfo(txInfo, zk.getData(getTXPath(fateId) + "/" + txInfo)); + return ZooStore.this.getTransactionInfo(txInfo, fateId); } @Override @@ -309,9 +309,9 @@ public List> getStack() { } } - private Serializable getTransactionInfo(TxInfo txInfo, long tid) { + private Serializable getTransactionInfo(TxInfo txInfo, FateId fateId) { try { - return deserializeTxInfo(txInfo, zk.getData(getTXPath(tid) + "/" + txInfo)); + return deserializeTxInfo(txInfo, zk.getData(getTXPath(fateId) + "/" + txInfo)); } catch (NoNodeException nne) { return null; } catch (KeeperException | InterruptedException e) { @@ -331,8 +331,8 @@ protected TStatus _getStatus(FateId fateId) { } @Override - protected Optional getKey(long tid) { - return Optional.ofNullable((byte[]) getTransactionInfo(TxInfo.TX_KEY, tid)); + protected Optional getKey(FateId fateId) { + return Optional.ofNullable((byte[]) getTransactionInfo(TxInfo.TX_KEY, fateId)); } @Override @@ -340,6 +340,11 @@ protected FateTxStore newFateTxStore(FateId fateId, boolean isReserved) { return new FateTxStoreImpl(fateId, isReserved); } + @Override + protected FateInstanceType getInstanceType() { + return fateInstanceType; + } + @Override protected Stream getTransactions() { try { diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index a988a2b5ab0..21d69e09198 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -37,7 +37,7 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.Fate.TxInfo; -import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; +import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; @@ -48,8 +48,8 @@ import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.ColumnFQ; -import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.hadoop.io.Text; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -117,9 +117,9 @@ public FateId getFateId() { } @Override - protected void create(long tid, byte[] key) { + protected void create(FateId fateId, byte[] key) { // TODO: conditional mutation should be used to verify tid is new - newMutator(tid).putStatus(TStatus.NEW).putKey(key).putCreateTime(System.currentTimeMillis()) + newMutator(fateId).putStatus(TStatus.NEW).putKey(key).putCreateTime(System.currentTimeMillis()) .mutate(); } @@ -155,18 +155,18 @@ protected TStatus _getStatus(FateId fateId) { } @Override - protected Optional getKey(long tid) { + protected Optional getKey(FateId fateId) { return scanTx(scanner -> { - scanner.setRange(getRow(tid)); + scanner.setRange(getRow(fateId)); TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); return scanner.stream().map(e -> e.getValue().get()).findFirst(); }); } @Override - protected Pair> getStatusAndKey(long tid) { + protected Pair> getStatusAndKey(FateId fateId) { return scanTx(scanner -> { - scanner.setRange(getRow(tid)); + scanner.setRange(getRow(fateId)); TxColumnFamily.STATUS_COLUMN.fetch(scanner); TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); @@ -197,6 +197,11 @@ protected FateTxStore newFateTxStore(FateId fateId, boolean isReserved) { return new FateTxStoreImpl(fateId, isReserved); } + @Override + protected FateInstanceType getInstanceType() { + return fateInstanceType; + } + static Range getRow(FateId fateId) { return new Range("tx_" + fateId.getHexTid()); } 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 e1fe65b27b2..d94abbf7e5b 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 @@ -143,12 +143,12 @@ public boolean isDeferredOverflow() { } @Override - public long create(byte[] key) { - long tid = store.create(key); + public FateId create(byte[] key) { + FateId fateId = store.create(key); if (storeLog.isTraceEnabled()) { - storeLog.trace("{} created fate transaction", formatTid(tid)); + storeLog.trace("{} created fate transaction", fateId); } - return tid; + return fateId; } }; } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 4160fcabc60..551a91913d9 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -47,28 +47,28 @@ public class TestStore implements FateStore { private final Map> txInfos = new HashMap<>(); private final Set reserved = new HashSet<>(); private static final FateInstanceType fateInstanceType = FateInstanceType.USER; - private Map> txInfos = new HashMap<>(); @Override public FateId create() { FateId fateId = FateId.from(fateInstanceType, nextId++); - statuses.put(nextId, new Pair<>(TStatus.NEW, Optional.empty())); + statuses.put(fateId, new Pair<>(TStatus.NEW, Optional.empty())); return fateId; } @Override - public long create(byte[] key) { + public FateId create(byte[] key) { HashCode hashCode = Hashing.murmur3_128().hashBytes(key); long tid = hashCode.asLong() & 0x7fffffffffffffffL; - if (statuses.putIfAbsent(tid, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { - throw new IllegalStateException("Transaction with tid " + tid + " already exists"); + FateId fateId = FateId.from(fateInstanceType, tid); + if (statuses.putIfAbsent(fateId, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { + throw new IllegalStateException("Transaction with fateId " + fateId + " already exists"); } - return tid; + return fateId; } - @Override @Override + @Override public FateTxStore reserve(FateId fateId) { - if (reserved.contains(fateId)) { @Override + if (reserved.contains(fateId)) { throw new IllegalStateException(); // zoo store would wait, but do not expect test to reserve } // twice... if test change, then change this @@ -114,7 +114,7 @@ public TStatus getStatus() { public Optional getKey() { return getStatusAndKey().getSecond(); } - + @Override public Pair> getStatusAndKey() { if (!reserved.contains(fateId)) { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index 021dc5df793..7b47251580c 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -229,8 +229,8 @@ protected void testDeferredOverflow(FateStore store, ServerContext sctx executor.shutdownNow(); // Cleanup so we don't interfere with other tests // All stores should already be unreserved - store.list() - .forEach(fateIdStatus -> store.tryReserve(fateIdStatus.getFateId()).orElseThrow().delete()); + store.list().forEach( + fateIdStatus -> store.tryReserve(fateIdStatus.getFateId()).orElseThrow().delete()); } } @@ -244,14 +244,14 @@ protected void testCreateWithKey(FateStore store, ServerContext sctx) { KeyExtent ke2 = new KeyExtent(TableId.of("tableId2"), new Text("zzz"), new Text("aaa")); byte[] key1 = serialize(ke1); - long tid1 = store.create(key1); + FateId fateId1 = store.create(key1); byte[] key2 = serialize(ke2); - long tid2 = store.create(key2); - assertNotEquals(tid1, tid2); + FateId fateId2 = store.create(key2); + assertNotEquals(fateId1, fateId2); - FateTxStore txStore1 = store.reserve(tid1); - FateTxStore txStore2 = store.reserve(tid2); + FateTxStore txStore1 = store.reserve(fateId1); + FateTxStore txStore2 = store.reserve(fateId2); try { assertTrue(txStore1.timeCreated() > 0); assertEquals(TStatus.NEW, txStore1.getStatus()); @@ -279,11 +279,11 @@ protected void testCreateWithKeyDuplicate(FateStore store, ServerContex // Creating with the same key should be fine if the status is NEW // It should just return the same id and allow us to continue reserving byte[] key = serialize(ke); - long tid1 = store.create(key); - long tid2 = store.create(key); - assertEquals(tid1, tid2); + FateId fateId1 = store.create(key); + FateId fateId2 = store.create(key); + assertEquals(fateId1, fateId2); - FateTxStore txStore = store.reserve(tid1); + FateTxStore txStore = store.reserve(fateId1); try { assertTrue(txStore.timeCreated() > 0); assertEquals(TStatus.NEW, txStore.getStatus()); @@ -303,9 +303,9 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); byte[] key = serialize(ke); - long tid1 = store.create(key); + FateId fateId1 = store.create(key); - FateTxStore txStore = store.reserve(tid1); + FateTxStore txStore = store.reserve(fateId1); try { assertTrue(txStore.timeCreated() > 0); txStore.setStatus(TStatus.IN_PROGRESS); From ec63c8bbc69cc3e9acec426403b44925487c88c1 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 2 Feb 2024 11:58:37 -0500 Subject: [PATCH 03/16] Create FateKey instead of a byte array for tracking the key for a fate operation --- .../accumulo/core/fate/AbstractFateStore.java | 25 ++- .../apache/accumulo/core/fate/FateKey.java | 162 ++++++++++++++++++ .../apache/accumulo/core/fate/FateStore.java | 2 +- .../accumulo/core/fate/ReadOnlyFateStore.java | 4 +- .../core/fate/WrappedFateTxStore.java | 4 +- .../apache/accumulo/core/fate/ZooStore.java | 13 +- .../core/fate/accumulo/AccumuloStore.java | 17 +- .../core/fate/accumulo/FateMutator.java | 2 +- .../core/fate/accumulo/FateMutatorImpl.java | 5 +- .../accumulo/core/logging/FateLogger.java | 5 +- .../apache/accumulo/core/fate/TestStore.java | 14 +- .../test/fate/accumulo/FateStoreIT.java | 47 ++--- 12 files changed, 226 insertions(+), 74 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/fate/FateKey.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 1ea96d56e45..1ca36119538 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -27,7 +27,6 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.io.UncheckedIOException; -import java.util.Arrays; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -244,24 +243,24 @@ public int getDeferredCount() { } @Override - public FateId create(byte[] key) { - HashCode hashCode = Hashing.murmur3_128().hashBytes(key); + public FateId create(FateKey fateKey) { + HashCode hashCode = Hashing.murmur3_128().hashBytes(fateKey.getSerialized()); long tid = hashCode.asLong() & 0x7fffffffffffffffL; FateId fateId = FateId.from(getInstanceType(), tid); - Pair> statusAndKey = getStatusAndKey(fateId); + Pair> statusAndKey = getStatusAndKey(fateId); TStatus status = statusAndKey.getFirst(); - Optional tKey = statusAndKey.getSecond(); + Optional tFateKey = statusAndKey.getSecond(); // Case 1: Status of UNKNOWN means doesn't exist, so we can create if (status == TStatus.UNKNOWN) { - create(fateId, key); + create(fateId, fateKey); // Case 2: Status is NEW so this is unseeded, we can return and allow the calling code // to reserve/seed as long as the existing key is the same and not different as that would // mean a collision } else if (status == TStatus.NEW) { - Preconditions.checkState(tKey.isPresent(), "Tx key column is missing"); - Preconditions.checkState(Arrays.equals(key, tKey.orElseThrow()), + Preconditions.checkState(tFateKey.isPresent(), "Tx key column is missing"); + Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), "Collision detected for tid %s", tid); // Case 3: Status is some other state which means already in progress } else { @@ -271,15 +270,15 @@ public FateId create(byte[] key) { return fateId; } - protected abstract void create(FateId fateId, byte[] key); + protected abstract void create(FateId fateId, FateKey fateKey); - protected abstract Pair> getStatusAndKey(FateId fateId); + protected abstract Pair> getStatusAndKey(FateId fateId); protected abstract Stream getTransactions(); protected abstract TStatus _getStatus(FateId fateId); - protected abstract Optional getKey(FateId fateId); + protected abstract Optional getKey(FateId fateId); protected abstract FateTxStore newFateTxStore(FateId fateId, boolean isReserved); @@ -378,13 +377,13 @@ public TStatus getStatus() { } @Override - public Optional getKey() { + public Optional getKey() { verifyReserved(false); return AbstractFateStore.this.getKey(fateId); } @Override - public Pair> getStatusAndKey() { + public Pair> getStatusAndKey() { verifyReserved(false); return AbstractFateStore.this.getStatusAndKey(fateId); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java b/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java new file mode 100644 index 00000000000..2b4238d087e --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateKey.java @@ -0,0 +1,162 @@ +/* + * 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.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; + +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.hadoop.io.DataInputBuffer; + +public class FateKey { + + private final FateKeyType type; + private final Optional keyExtent; + private final Optional compactionId; + private final byte[] serialized; + + private FateKey(FateKeyType type, KeyExtent keyExtent) { + this.type = Objects.requireNonNull(type); + this.keyExtent = Optional.of(keyExtent); + this.compactionId = Optional.empty(); + this.serialized = serialize(type, keyExtent); + } + + private FateKey(FateKeyType type, ExternalCompactionId compactionId) { + this.type = Objects.requireNonNull(type); + this.keyExtent = Optional.empty(); + this.compactionId = Optional.of(compactionId); + this.serialized = serialize(type, compactionId); + } + + private FateKey(byte[] serialized) { + try (DataInputBuffer buffer = new DataInputBuffer()) { + buffer.reset(serialized, serialized.length); + this.type = FateKeyType.valueOf(buffer.readUTF()); + this.keyExtent = deserializeKeyExtent(type, buffer); + this.compactionId = deserializeCompactionId(type, buffer); + this.serialized = serialized; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public FateKeyType getType() { + return type; + } + + public Optional getKeyExtent() { + return keyExtent; + } + + public Optional getCompactionId() { + return compactionId; + } + + public byte[] getSerialized() { + return serialized; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FateKey fateKey = (FateKey) o; + return Arrays.equals(serialized, fateKey.serialized); + } + + @Override + public int hashCode() { + return Arrays.hashCode(serialized); + } + + public static FateKey deserialize(byte[] serialized) { + return new FateKey(serialized); + } + + public static FateKey forSplit(KeyExtent extent) { + return new FateKey(FateKeyType.SPLIT, extent); + } + + public static FateKey forCompactionCommit(ExternalCompactionId compactionId) { + return new FateKey(FateKeyType.COMPACTION_COMMIT, compactionId); + } + + public enum FateKeyType { + SPLIT, COMPACTION_COMMIT + } + + private static byte[] serialize(FateKeyType type, KeyExtent ke) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos)) { + dos.writeUTF(type.toString()); + ke.writeTo(dos); + dos.close(); + return baos.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static byte[] serialize(FateKeyType type, ExternalCompactionId compactionId) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos)) { + dos.writeUTF(type.toString()); + dos.writeUTF(compactionId.canonical()); + dos.close(); + return baos.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static Optional deserializeKeyExtent(FateKeyType type, DataInputBuffer buffer) + throws IOException { + switch (type) { + case SPLIT: + return Optional.of(KeyExtent.readFrom(buffer)); + case COMPACTION_COMMIT: + return Optional.empty(); + default: + throw new IllegalStateException("Unexpected FateInstanceType found " + type); + } + } + + private static Optional deserializeCompactionId(FateKeyType type, + DataInputBuffer buffer) throws IOException { + switch (type) { + case SPLIT: + return Optional.empty(); + case COMPACTION_COMMIT: + return Optional.of(ExternalCompactionId.of(buffer.readUTF())); + default: + throw new IllegalStateException("Unexpected FateInstanceType found " + type); + } + } +} 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 6c91d8d11ab..7efee2267fa 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 @@ -39,7 +39,7 @@ public interface FateStore extends ReadOnlyFateStore { */ FateId create(); - FateId create(byte[] key); + FateId create(FateKey fateKey); /** * An interface that allows read/write access to the data related to a single fate operation. 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 b604936d274..deb79413c92 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 @@ -90,9 +90,9 @@ interface ReadOnlyFateTxStore { */ TStatus getStatus(); - Optional getKey(); + Optional getKey(); - Pair> getStatusAndKey(); + Pair> getStatusAndKey(); /** * Wait for the status of a transaction to change diff --git a/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java b/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java index e898c263162..031a3ece02b 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/WrappedFateTxStore.java @@ -60,12 +60,12 @@ public FateStore.TStatus getStatus() { } @Override - public Optional getKey() { + public Optional getKey() { return wrapped.getKey(); } @Override - public Pair> getStatusAndKey() { + public Pair> getStatusAndKey() { return wrapped.getStatusAndKey(); } 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 b48fb189cb4..22a71ecc62f 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 @@ -96,7 +96,7 @@ public FateId create() { } @Override - protected void create(FateId fateId, byte[] key) { + protected void create(FateId fateId, FateKey key) { // TODO: Should we somehow make this Atomic or clean up on failure to make sure // that either both of these writes happen or none happen? try { @@ -104,15 +104,15 @@ protected void create(FateId fateId, byte[] key) { NodeExistsPolicy.FAIL); // The key was already used to generate the tid but we still need to store it // separate to check later for collision detection - zk.putPersistentData(getTXPath(fateId) + "/" + TxInfo.TX_KEY, serializeTxInfo(key), - NodeExistsPolicy.OVERWRITE); + zk.putPersistentData(getTXPath(fateId) + "/" + TxInfo.TX_KEY, + serializeTxInfo(key.getSerialized()), NodeExistsPolicy.OVERWRITE); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } } @Override - protected Pair> getStatusAndKey(FateId fateId) { + protected Pair> getStatusAndKey(FateId fateId) { return new Pair<>(_getStatus(fateId), getKey(fateId)); } @@ -331,8 +331,9 @@ protected TStatus _getStatus(FateId fateId) { } @Override - protected Optional getKey(FateId fateId) { - return Optional.ofNullable((byte[]) getTransactionInfo(TxInfo.TX_KEY, fateId)); + protected Optional getKey(FateId fateId) { + return Optional.ofNullable((byte[]) getTransactionInfo(TxInfo.TX_KEY, fateId)) + .map(FateKey::deserialize); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index 21d69e09198..91fc84c2f8a 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -39,6 +39,7 @@ import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; @@ -117,10 +118,10 @@ public FateId getFateId() { } @Override - protected void create(FateId fateId, byte[] key) { + protected void create(FateId fateId, FateKey fateKey) { // TODO: conditional mutation should be used to verify tid is new - newMutator(fateId).putStatus(TStatus.NEW).putKey(key).putCreateTime(System.currentTimeMillis()) - .mutate(); + newMutator(fateId).putStatus(TStatus.NEW).putKey(fateKey.getSerialized()) + .putCreateTime(System.currentTimeMillis()).mutate(); } @Override @@ -155,23 +156,23 @@ protected TStatus _getStatus(FateId fateId) { } @Override - protected Optional getKey(FateId fateId) { + protected Optional getKey(FateId fateId) { return scanTx(scanner -> { scanner.setRange(getRow(fateId)); TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); - return scanner.stream().map(e -> e.getValue().get()).findFirst(); + return scanner.stream().map(e -> FateKey.deserialize(e.getValue().get())).findFirst(); }); } @Override - protected Pair> getStatusAndKey(FateId fateId) { + protected Pair> getStatusAndKey(FateId fateId) { return scanTx(scanner -> { scanner.setRange(getRow(fateId)); TxColumnFamily.STATUS_COLUMN.fetch(scanner); TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); TStatus status = null; - byte[] key = null; + FateKey key = null; for (Entry entry : scanner) { final String qual = entry.getKey().getColumnQualifierData().toString(); @@ -180,7 +181,7 @@ protected Pair> getStatusAndKey(FateId fateId) { status = TStatus.valueOf(entry.getValue().toString()); break; case TxInfoColumnFamily.TX_KEY: - key = entry.getValue().get(); + key = FateKey.deserialize(entry.getValue().get()); break; default: throw new IllegalStateException("Unexpected column qualifier: " + qual); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java index def701dd9d8..5b7527778c4 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java @@ -38,7 +38,7 @@ public interface FateMutator { FateMutator putAgeOff(byte[] data); - FateMutator putKey(byte[] data); + FateMutator putKey(byte[] fateKey); FateMutator putTxInfo(Fate.TxInfo txInfo, byte[] data); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java index 8195e75df0e..80ad27f774b 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java @@ -101,9 +101,10 @@ public FateMutator putAgeOff(byte[] data) { return this; } + // TODO replace with FateKey @Override - public FateMutator putKey(byte[] data) { - TxInfoColumnFamily.TX_KEY_COLUMN.put(mutation, new Value(data)); + public FateMutator putKey(byte[] fateKey) { + TxInfoColumnFamily.TX_KEY_COLUMN.put(mutation, new Value(fateKey)); return this; } 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 d94abbf7e5b..bdca78ae0c9 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 @@ -27,6 +27,7 @@ import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.FateStore.FateTxStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore; @@ -143,8 +144,8 @@ public boolean isDeferredOverflow() { } @Override - public FateId create(byte[] key) { - FateId fateId = store.create(key); + public FateId create(FateKey fateKey) { + FateId fateId = store.create(fateKey); if (storeLog.isTraceEnabled()) { storeLog.trace("{} created fate transaction", fateId); } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 551a91913d9..627b60e3ccd 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -43,7 +43,7 @@ public class TestStore implements FateStore { private long nextId = 1; - private final Map>> statuses = new HashMap<>(); + private final Map>> statuses = new HashMap<>(); private final Map> txInfos = new HashMap<>(); private final Set reserved = new HashSet<>(); private static final FateInstanceType fateInstanceType = FateInstanceType.USER; @@ -56,8 +56,8 @@ public FateId create() { } @Override - public FateId create(byte[] key) { - HashCode hashCode = Hashing.murmur3_128().hashBytes(key); + public FateId create(FateKey key) { + HashCode hashCode = Hashing.murmur3_128().hashBytes(key.getSerialized()); long tid = hashCode.asLong() & 0x7fffffffffffffffL; FateId fateId = FateId.from(fateInstanceType, tid); if (statuses.putIfAbsent(fateId, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { @@ -111,17 +111,17 @@ public TStatus getStatus() { } @Override - public Optional getKey() { + public Optional getKey() { return getStatusAndKey().getSecond(); } @Override - public Pair> getStatusAndKey() { + public Pair> getStatusAndKey() { if (!reserved.contains(fateId)) { throw new IllegalStateException(); } - Pair> status = statuses.get(fateId); + Pair> status = statuses.get(fateId); if (status == null) { return new Pair<>(TStatus.UNKNOWN, Optional.empty()); } @@ -169,7 +169,7 @@ public void setStatus(TStatus status) { if (!reserved.contains(fateId)) { throw new IllegalStateException(); } - Pair> currentStatus = statuses.get(fateId); + Pair> currentStatus = statuses.get(fateId); if (currentStatus == null) { throw new IllegalStateException(); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index 7b47251580c..11903ef0105 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.fate.accumulo; -import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -28,14 +27,11 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.UncheckedIOException; import java.time.Duration; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.UUID; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -45,11 +41,13 @@ import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.core.fate.FateStore.FateTxStore; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.ReadOnlyRepo; import org.apache.accumulo.core.fate.StackOverflowException; +import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateIT.TestRepo; @@ -241,13 +239,13 @@ public void testCreateWithKey() throws Exception { protected void testCreateWithKey(FateStore store, ServerContext sctx) { KeyExtent ke1 = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); - KeyExtent ke2 = new KeyExtent(TableId.of("tableId2"), new Text("zzz"), new Text("aaa")); - byte[] key1 = serialize(ke1); - FateId fateId1 = store.create(key1); + FateKey fateKey1 = FateKey.forSplit(ke1); + FateId fateId1 = store.create(fateKey1); - byte[] key2 = serialize(ke2); - FateId fateId2 = store.create(key2); + FateKey fateKey2 = + FateKey.forCompactionCommit(ExternalCompactionId.generate(UUID.randomUUID())); + FateId fateId2 = store.create(fateKey2); assertNotEquals(fateId1, fateId2); FateTxStore txStore1 = store.reserve(fateId1); @@ -255,11 +253,11 @@ protected void testCreateWithKey(FateStore store, ServerContext sctx) { try { assertTrue(txStore1.timeCreated() > 0); assertEquals(TStatus.NEW, txStore1.getStatus()); - assertArrayEquals(key1, txStore1.getKey().orElseThrow()); + assertEquals(fateKey1, txStore1.getKey().orElseThrow()); assertTrue(txStore2.timeCreated() > 0); assertEquals(TStatus.NEW, txStore2.getStatus()); - assertArrayEquals(key2, txStore2.getKey().orElseThrow()); + assertEquals(fateKey2, txStore2.getKey().orElseThrow()); assertEquals(2, store.list().count()); } finally { @@ -278,16 +276,16 @@ protected void testCreateWithKeyDuplicate(FateStore store, ServerContex // Creating with the same key should be fine if the status is NEW // It should just return the same id and allow us to continue reserving - byte[] key = serialize(ke); - FateId fateId1 = store.create(key); - FateId fateId2 = store.create(key); + FateKey fateKey = FateKey.forSplit(ke); + FateId fateId1 = store.create(fateKey); + FateId fateId2 = store.create(fateKey); assertEquals(fateId1, fateId2); FateTxStore txStore = store.reserve(fateId1); try { assertTrue(txStore.timeCreated() > 0); assertEquals(TStatus.NEW, txStore.getStatus()); - assertArrayEquals(key, txStore.getKey().orElseThrow()); + assertEquals(fateKey, txStore.getKey().orElseThrow()); assertEquals(1, store.list().count()); } finally { txStore.delete(); @@ -302,8 +300,8 @@ public void testCreateWithKeyInProgress() throws Exception { protected void testCreateWithKeyInProgress(FateStore store, ServerContext sctx) { KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); - byte[] key = serialize(ke); - FateId fateId1 = store.create(key); + FateKey fateKey = FateKey.forSplit(ke); + FateId fateId1 = store.create(fateKey); FateTxStore txStore = store.reserve(fateId1); try { @@ -312,23 +310,12 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte // We have an existing transaction with the same key in progress // so should not be allowed - assertThrows(IllegalStateException.class, () -> store.create(key)); + assertThrows(IllegalStateException.class, () -> store.create(fateKey)); } finally { txStore.delete(); } } - private byte[] serialize(KeyExtent ke) { - try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dos = new DataOutputStream(baos)) { - ke.writeTo(dos); - dos.close(); - return baos.toByteArray(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - private static class TestOperation2 extends TestRepo { private static final long serialVersionUID = 1L; From 1646242b5a50883e1a8dad21631cc42004105974 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 2 Feb 2024 12:40:48 -0500 Subject: [PATCH 04/16] Make FateKey atomic This commit moves FateKey storage to the node for the ZooStore so it will be inserted atomically with the transaction id and also moves it to the TxColumnFamily for the Accumulo store. --- .../org/apache/accumulo/core/fate/Fate.java | 2 +- .../apache/accumulo/core/fate/ZooStore.java | 91 +++++++++++++++---- .../core/fate/accumulo/AccumuloStore.java | 19 ++-- .../core/fate/accumulo/FateMutator.java | 5 +- .../core/fate/accumulo/FateMutatorImpl.java | 17 ++-- .../core/fate/accumulo/schema/FateSchema.java | 6 +- .../test/fate/zookeeper/ZookeeperFateIT.java | 14 ++- 7 files changed, 107 insertions(+), 47 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 3de803195db..938b76ef4c2 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 @@ -77,7 +77,7 @@ public class Fate { private final Thread workFinder; public enum TxInfo { - TX_NAME, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE, TX_KEY + TX_NAME, AUTO_CLEAN, EXCEPTION, TX_AGEOFF, RETURN_VALUE } /** 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 22a71ecc62f..9e329876c5a 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 @@ -19,14 +19,18 @@ 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.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; import java.io.Serializable; +import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.function.Supplier; import java.util.stream.Stream; @@ -36,6 +40,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.util.Pair; +import org.apache.hadoop.io.DataInputBuffer; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; @@ -84,7 +89,7 @@ public FateId create() { // looking at the code for SecureRandom, it appears to be thread safe long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL; FateId fateId = FateId.from(fateInstanceType, tid); - zk.putPersistentData(getTXPath(fateId), TStatus.NEW.name().getBytes(UTF_8), + zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW).serialize(), NodeExistsPolicy.FAIL); return fateId; } catch (NodeExistsException nee) { @@ -97,15 +102,9 @@ public FateId create() { @Override protected void create(FateId fateId, FateKey key) { - // TODO: Should we somehow make this Atomic or clean up on failure to make sure - // that either both of these writes happen or none happen? try { - zk.putPersistentData(getTXPath(fateId), TStatus.NEW.name().getBytes(UTF_8), + zk.putPersistentData(getTXPath(fateId), new NodeValue(TStatus.NEW, key).serialize(), NodeExistsPolicy.FAIL); - // The key was already used to generate the tid but we still need to store it - // separate to check later for collision detection - zk.putPersistentData(getTXPath(fateId) + "/" + TxInfo.TX_KEY, - serializeTxInfo(key.getSerialized()), NodeExistsPolicy.OVERWRITE); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -113,7 +112,8 @@ protected void create(FateId fateId, FateKey key) { @Override protected Pair> getStatusAndKey(FateId fateId) { - return new Pair<>(_getStatus(fateId), getKey(fateId)); + final NodeValue node = getNode(fateId); + return new Pair<>(node.status, node.fateKey); } private class FateTxStoreImpl extends AbstractFateTxStoreImpl { @@ -216,7 +216,7 @@ public void setStatus(TStatus status) { verifyReserved(true); try { - zk.putPersistentData(getTXPath(fateId), status.name().getBytes(UTF_8), + zk.putPersistentData(getTXPath(fateId), new NodeValue(status).serialize(), NodeExistsPolicy.OVERWRITE); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); @@ -321,21 +321,24 @@ private Serializable getTransactionInfo(TxInfo txInfo, FateId fateId) { @Override protected TStatus _getStatus(FateId fateId) { + return getNode(fateId).status; + } + + @Override + protected Optional getKey(FateId fateId) { + return getNode(fateId).fateKey; + } + + private NodeValue getNode(FateId fateId) { try { - return TStatus.valueOf(new String(zk.getData(getTXPath(fateId)), UTF_8)); + return new NodeValue(zk.getData(getTXPath(fateId))); } catch (NoNodeException nne) { - return TStatus.UNKNOWN; + return new NodeValue(TStatus.UNKNOWN); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } } - @Override - protected Optional getKey(FateId fateId) { - return Optional.ofNullable((byte[]) getTransactionInfo(TxInfo.TX_KEY, fateId)) - .map(FateKey::deserialize); - } - @Override protected FateTxStore newFateTxStore(FateId fateId, boolean isReserved) { return new FateTxStoreImpl(fateId, isReserved); @@ -367,4 +370,54 @@ public TStatus getStatus() { } } + protected static class NodeValue { + final TStatus status; + final Optional fateKey; + + private NodeValue(byte[] serialized) { + try (DataInputBuffer buffer = new DataInputBuffer()) { + buffer.reset(serialized, serialized.length); + this.status = TStatus.valueOf(buffer.readUTF()); + this.fateKey = deserializeFateKey(buffer); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private NodeValue(TStatus status) { + this(status, null); + } + + private NodeValue(TStatus status, FateKey fateKey) { + this.status = Objects.requireNonNull(status); + this.fateKey = Optional.ofNullable(fateKey); + } + + private Optional deserializeFateKey(DataInputBuffer buffer) throws IOException { + int length = buffer.readInt(); + if (length > 0) { + return Optional.of(FateKey.deserialize(buffer.readNBytes(length))); + } + return Optional.empty(); + } + + byte[] serialize() { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos)) { + dos.writeUTF(status.name()); + if (fateKey.isPresent()) { + byte[] serialized = fateKey.orElseThrow().getSerialized(); + dos.writeInt(serialized.length); + dos.write(serialized); + } else { + dos.writeInt(0); + } + dos.close(); + return baos.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + } } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index 91fc84c2f8a..064f06b168b 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -43,6 +43,7 @@ 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.accumulo.FateMutator.Status; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.RepoColumnFamily; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxColumnFamily; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxInfoColumnFamily; @@ -119,9 +120,12 @@ public FateId getFateId() { @Override protected void create(FateId fateId, FateKey fateKey) { - // TODO: conditional mutation should be used to verify tid is new - newMutator(fateId).putStatus(TStatus.NEW).putKey(fateKey.getSerialized()) - .putCreateTime(System.currentTimeMillis()).mutate(); + var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) + .putCreateTime(System.currentTimeMillis()).tryMutate(); + + // TODO: Any reason to retry here? + Preconditions.checkState(status == Status.ACCEPTED, + "" + "Failed to create transaction with fateId %s and fateKey %s", fateId, fateKey); } @Override @@ -159,7 +163,7 @@ protected TStatus _getStatus(FateId fateId) { protected Optional getKey(FateId fateId) { return scanTx(scanner -> { scanner.setRange(getRow(fateId)); - TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); + TxColumnFamily.TX_KEY_COLUMN.fetch(scanner); return scanner.stream().map(e -> FateKey.deserialize(e.getValue().get())).findFirst(); }); } @@ -169,7 +173,7 @@ protected Pair> getStatusAndKey(FateId fateId) { return scanTx(scanner -> { scanner.setRange(getRow(fateId)); TxColumnFamily.STATUS_COLUMN.fetch(scanner); - TxInfoColumnFamily.TX_KEY_COLUMN.fetch(scanner); + TxColumnFamily.TX_KEY_COLUMN.fetch(scanner); TStatus status = null; FateKey key = null; @@ -180,7 +184,7 @@ protected Pair> getStatusAndKey(FateId fateId) { case TxColumnFamily.STATUS: status = TStatus.valueOf(entry.getValue().toString()); break; - case TxInfoColumnFamily.TX_KEY: + case TxColumnFamily.TX_KEY: key = FateKey.deserialize(entry.getValue().get()); break; default: @@ -280,9 +284,6 @@ public Serializable getTransactionInfo(TxInfo txInfo) { case TX_AGEOFF: cq = TxInfoColumnFamily.TX_AGEOFF_COLUMN; break; - case TX_KEY: - cq = TxInfoColumnFamily.TX_KEY_COLUMN; - break; default: throw new IllegalArgumentException("Unexpected TxInfo type " + txInfo); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java index 5b7527778c4..b538a9f665d 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutator.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.fate.accumulo; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; @@ -26,6 +27,8 @@ public interface FateMutator { FateMutator putStatus(TStatus status); + FateMutator putKey(FateKey fateKey); + FateMutator putCreateTime(long ctime); FateMutator putName(byte[] data); @@ -38,8 +41,6 @@ public interface FateMutator { FateMutator putAgeOff(byte[] data); - FateMutator putKey(byte[] fateKey); - FateMutator putTxInfo(Fate.TxInfo txInfo, byte[] data); FateMutator putRepo(int position, Repo repo); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java index 80ad27f774b..3cc31bb6fcd 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java @@ -37,6 +37,7 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.FateKey; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.RepoColumnFamily; @@ -65,6 +66,12 @@ public FateMutator putStatus(TStatus status) { return this; } + @Override + public FateMutator putKey(FateKey fateKey) { + TxColumnFamily.TX_KEY_COLUMN.put(mutation, new Value(fateKey.getSerialized())); + return this; + } + @Override public FateMutator putCreateTime(long ctime) { TxColumnFamily.CREATE_TIME_COLUMN.put(mutation, new Value(Long.toString(ctime))); @@ -101,13 +108,6 @@ public FateMutator putAgeOff(byte[] data) { return this; } - // TODO replace with FateKey - @Override - public FateMutator putKey(byte[] fateKey) { - TxInfoColumnFamily.TX_KEY_COLUMN.put(mutation, new Value(fateKey)); - return this; - } - @Override public FateMutator putTxInfo(TxInfo txInfo, byte[] data) { switch (txInfo) { @@ -126,9 +126,6 @@ public FateMutator putTxInfo(TxInfo txInfo, byte[] data) { case TX_AGEOFF: putAgeOff(data); break; - case TX_KEY: - putKey(data); - break; default: throw new IllegalArgumentException("Unexpected TxInfo type: " + txInfo); } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java index 41205dd1f4a..73664f7a1b4 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/schema/FateSchema.java @@ -30,6 +30,9 @@ public static class TxColumnFamily { public static final String STATUS = "status"; public static final ColumnFQ STATUS_COLUMN = new ColumnFQ(NAME, new Text(STATUS)); + public static final String TX_KEY = "txkey"; + public static final ColumnFQ TX_KEY_COLUMN = new ColumnFQ(NAME, new Text(TX_KEY)); + public static final String CREATE_TIME = "ctime"; public static final ColumnFQ CREATE_TIME_COLUMN = new ColumnFQ(NAME, new Text(CREATE_TIME)); } @@ -41,9 +44,6 @@ public static class TxInfoColumnFamily { public static final String TX_NAME = "txname"; public static final ColumnFQ TX_NAME_COLUMN = new ColumnFQ(NAME, new Text(TX_NAME)); - public static final String TX_KEY = "txkey"; - public static final ColumnFQ TX_KEY_COLUMN = new ColumnFQ(NAME, new Text(TX_KEY)); - public static final String AUTO_CLEAN = "autoclean"; public static final ColumnFQ AUTO_CLEAN_COLUMN = new ColumnFQ(NAME, new Text(AUTO_CLEAN)); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java index 089df59de5c..18de2ef9221 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java @@ -18,13 +18,14 @@ */ package org.apache.accumulo.test.fate.zookeeper; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; import java.util.UUID; import org.apache.accumulo.core.Constants; @@ -35,6 +36,7 @@ import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateIT; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; +import org.apache.hadoop.io.DataInputBuffer; import org.apache.zookeeper.KeeperException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -91,11 +93,17 @@ private static TStatus getTxStatus(ZooReaderWriter zrw, FateId fateId) throws KeeperException, InterruptedException { zrw.sync(ZK_ROOT); String txdir = String.format("%s%s/tx_%s", ZK_ROOT, Constants.ZFATE, fateId.getHexTid()); - try { - return TStatus.valueOf(new String(zrw.getData(txdir), UTF_8)); + + try (DataInputBuffer buffer = new DataInputBuffer()) { + var serialized = zrw.getData(txdir); + buffer.reset(serialized, serialized.length); + return TStatus.valueOf(buffer.readUTF()); + } catch (IOException e) { + throw new UncheckedIOException(e); } catch (KeeperException.NoNodeException e) { return TStatus.UNKNOWN; } + } } From fe62cfb0ef308f5c404ca9542bd14f2fe99e3d2b Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 2 Feb 2024 12:51:48 -0500 Subject: [PATCH 05/16] fix checkstyle --- .../org/apache/accumulo/core/fate/accumulo/AccumuloStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index 064f06b168b..140b9bb0bef 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -125,7 +125,7 @@ protected void create(FateId fateId, FateKey fateKey) { // TODO: Any reason to retry here? Preconditions.checkState(status == Status.ACCEPTED, - "" + "Failed to create transaction with fateId %s and fateKey %s", fateId, fateKey); + "Failed to create transaction with fateId %s and fateKey %s", fateId, fateKey); } @Override From 5824aa91d35b680c13d824065c6f5413f2e099d8 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Sun, 4 Feb 2024 14:50:36 -0500 Subject: [PATCH 06/16] Test updates and support configuring hashing algorithm for FateId from FateKey --- .../accumulo/core/fate/AbstractFateStore.java | 32 ++++++++--- .../apache/accumulo/core/fate/ZooStore.java | 8 ++- .../core/fate/accumulo/AccumuloStore.java | 9 ++- .../org/apache/accumulo/test/fate/FateIT.java | 3 +- .../accumulo/test/fate/FateTestRunner.java | 8 ++- .../test/fate/accumulo/AccumuloFateIT.java | 6 +- .../fate/accumulo/AccumuloStoreFateIT.java | 6 +- .../test/fate/accumulo/FateStoreIT.java | 56 ++++++++++++++++++- .../test/fate/zookeeper/ZooStoreFateIT.java | 7 ++- .../test/fate/zookeeper/ZookeeperFateIT.java | 7 ++- 10 files changed, 113 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 1ca36119538..bba31c529f5 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -31,6 +31,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -56,12 +57,22 @@ public abstract class AbstractFateStore implements FateStore { // Default maximum size of 100,000 transactions before deferral is stopped and // all existing transactions are processed immediately again - protected static final int DEFAULT_MAX_DEFERRED = 100_000; + public static final int DEFAULT_MAX_DEFERRED = 100_000; + + public static final FateIdGenerator DEFAULT_FATE_ID_GENERATOR = new FateIdGenerator() { + @Override + public FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey) { + HashCode hashCode = Hashing.murmur3_128().hashBytes(fateKey.getSerialized()); + long tid = hashCode.asLong() & 0x7fffffffffffffffL; + return FateId.from(instanceType, tid); + } + }; protected final Set reserved; protected final Map deferred; private final int maxDeferred; private final AtomicBoolean deferredOverflow = new AtomicBoolean(); + private final FateIdGenerator fateIdGenerator; // This is incremented each time a transaction was unreserved that was non new protected final SignalCount unreservedNonNewCount = new SignalCount(); @@ -70,11 +81,12 @@ public abstract class AbstractFateStore implements FateStore { protected final SignalCount unreservedRunnableCount = new SignalCount(); public AbstractFateStore() { - this(DEFAULT_MAX_DEFERRED); + this(DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } - public AbstractFateStore(int maxDeferred) { + public AbstractFateStore(int maxDeferred, FateIdGenerator fateIdGenerator) { this.maxDeferred = maxDeferred; + this.fateIdGenerator = Objects.requireNonNull(fateIdGenerator); this.reserved = new HashSet<>(); this.deferred = new HashMap<>(); } @@ -244,10 +256,7 @@ public int getDeferredCount() { @Override public FateId create(FateKey fateKey) { - HashCode hashCode = Hashing.murmur3_128().hashBytes(fateKey.getSerialized()); - long tid = hashCode.asLong() & 0x7fffffffffffffffL; - FateId fateId = FateId.from(getInstanceType(), tid); - + FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); Pair> statusAndKey = getStatusAndKey(fateId); TStatus status = statusAndKey.getFirst(); Optional tFateKey = statusAndKey.getSecond(); @@ -261,10 +270,11 @@ public FateId create(FateKey fateKey) { } else if (status == TStatus.NEW) { Preconditions.checkState(tFateKey.isPresent(), "Tx key column is missing"); Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), - "Collision detected for tid %s", tid); + "Collision detected for tid %s", fateId.getTid()); // Case 3: Status is some other state which means already in progress } else { - throw new IllegalStateException("Existing transaction already exists for: " + tid); + throw new IllegalStateException( + "Existing transaction already exists for: " + fateId.getTid()); } return fateId; @@ -394,6 +404,10 @@ public FateId getID() { } } + public interface FateIdGenerator { + FateId fromTypeAndKey(FateInstanceType instanceType, FateKey fateKey); + } + protected byte[] serializeTxInfo(Serializable so) { if (so instanceof String) { return ("S " + so).getBytes(UTF_8); 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 9e329876c5a..d0ef9600546 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 @@ -48,6 +48,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Suppliers; //TODO use zoocache? - ACCUMULO-1297 @@ -65,12 +66,13 @@ private String getTXPath(FateId fateId) { } public ZooStore(String path, ZooReaderWriter zk) throws KeeperException, InterruptedException { - this(path, zk, DEFAULT_MAX_DEFERRED); + this(path, zk, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } - public ZooStore(String path, ZooReaderWriter zk, int maxDeferred) + @VisibleForTesting + public ZooStore(String path, ZooReaderWriter zk, int maxDeferred, FateIdGenerator fateIdGenerator) throws KeeperException, InterruptedException { - super(maxDeferred); + super(maxDeferred, fateIdGenerator); this.path = path; this.zk = zk; diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index 140b9bb0bef..a70e1d97617 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -56,6 +56,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; public class AccumuloStore extends AbstractFateStore { @@ -71,11 +72,13 @@ public class AccumuloStore extends AbstractFateStore { com.google.common.collect.Range.closed(1, maxRepos); public AccumuloStore(ClientContext context, String tableName) { - this(context, tableName, DEFAULT_MAX_DEFERRED); + this(context, tableName, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } - public AccumuloStore(ClientContext context, String tableName, int maxDeferred) { - super(maxDeferred); + @VisibleForTesting + public AccumuloStore(ClientContext context, String tableName, int maxDeferred, + FateIdGenerator fateIdGenerator) { + super(maxDeferred, fateIdGenerator); this.context = Objects.requireNonNull(context); this.tableName = Objects.requireNonNull(tableName); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java index 9a98280b896..cf8726a0cd5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateIT.java @@ -36,6 +36,7 @@ import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.Fate; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateStore; @@ -288,7 +289,7 @@ public void testDeferredOverflow() throws Exception { // Set a maximum deferred map size of 10 transactions so that when the 11th // is seen the Fate store should clear the deferred map and mark // the flag as overflow so that all the deferred transactions will be run - executeTest(this::testDeferredOverflow, 10); + executeTest(this::testDeferredOverflow, 10, AbstractFateStore.DEFAULT_FATE_ID_GENERATOR); } protected void testDeferredOverflow(FateStore store, ServerContext sctx) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java b/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java index 15df2404dd9..244f7991116 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateTestRunner.java @@ -18,16 +18,20 @@ */ package org.apache.accumulo.test.fate; +import org.apache.accumulo.core.fate.AbstractFateStore; +import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; import org.apache.accumulo.core.fate.FateStore; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateTestRunner.TestEnv; public interface FateTestRunner { - void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception; + void executeTest(FateTestExecutor testMethod, int maxDeferred, FateIdGenerator fateIdGenerator) + throws Exception; default void executeTest(FateTestExecutor testMethod) throws Exception { - executeTest(testMethod, 100_000); + executeTest(testMethod, AbstractFateStore.DEFAULT_MAX_DEFERRED, + AbstractFateStore.DEFAULT_FATE_ID_GENERATOR); } interface FateTestExecutor { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java index 29cf5743b4a..3091f8c88fd 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloFateIT.java @@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.accumulo.AccumuloStore; @@ -51,12 +52,13 @@ public static void teardown() { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred, + FateIdGenerator fateIdGenerator) throws Exception { table = getUniqueNames(1)[0]; try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { client.tableOperations().create(table); - testMethod.execute(new AccumuloStore<>(client, table, maxDeferred), + testMethod.execute(new AccumuloStore<>(client, table, maxDeferred, fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java index 2daee958180..7e2f0d8dffe 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java @@ -20,6 +20,7 @@ import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; import org.apache.accumulo.core.fate.accumulo.AccumuloStore; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.junit.jupiter.api.AfterAll; @@ -38,12 +39,13 @@ public static void teardown() { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred, + FateIdGenerator fateIdGenerator) throws Exception { String table = getUniqueNames(1)[0]; try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { client.tableOperations().create(table); - testMethod.execute(new AccumuloStore<>(client, table, maxDeferred), + testMethod.execute(new AccumuloStore<>(client, table, maxDeferred, fateIdGenerator), getCluster().getServerContext()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index 11903ef0105..df3a4aac143 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -26,6 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.time.Duration; import java.util.HashSet; @@ -39,6 +40,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.Fate.TxInfo; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateKey; @@ -150,7 +152,7 @@ protected void testReadWriteTxInfo(FateStore store, ServerContext sctx) @Test public void testDeferredOverflow() throws Exception { - executeTest(this::testDeferredOverflow, 10); + executeTest(this::testDeferredOverflow, 10, AbstractFateStore.DEFAULT_FATE_ID_GENERATOR); } protected void testDeferredOverflow(FateStore store, ServerContext sctx) @@ -263,6 +265,8 @@ protected void testCreateWithKey(FateStore store, ServerContext sctx) { } finally { txStore1.delete(); txStore2.delete(); + txStore1.unreserve(0, TimeUnit.SECONDS); + txStore2.unreserve(0, TimeUnit.SECONDS); } } @@ -289,6 +293,7 @@ protected void testCreateWithKeyDuplicate(FateStore store, ServerContex assertEquals(1, store.list().count()); } finally { txStore.delete(); + txStore.unreserve(0, TimeUnit.SECONDS); } } @@ -301,9 +306,9 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); FateKey fateKey = FateKey.forSplit(ke); - FateId fateId1 = store.create(fateKey); + FateId fateId = store.create(fateKey); - FateTxStore txStore = store.reserve(fateId1); + FateTxStore txStore = store.reserve(fateId); try { assertTrue(txStore.timeCreated() > 0); txStore.setStatus(TStatus.IN_PROGRESS); @@ -313,7 +318,52 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte assertThrows(IllegalStateException.class, () -> store.create(fateKey)); } finally { txStore.delete(); + txStore.unreserve(0, TimeUnit.SECONDS); + } + + try { + // After deletion, make sure we can create again with the same key + fateId = store.create(fateKey); + txStore = store.reserve(fateId); + assertTrue(txStore.timeCreated() > 0); + assertEquals(TStatus.NEW, txStore.getStatus()); + } finally { + txStore.delete(); + txStore.unreserve(0, TimeUnit.SECONDS); } + + } + + @Test + public void testCreateWithKeyCollision() throws Exception { + // Replace the default hasing algorithm with one that always returns the same tid so + // we can check duplicate detection with different keys + executeTest(this::testCreateWithKeyCollision, AbstractFateStore.DEFAULT_MAX_DEFERRED, + (instanceType, fateKey) -> FateId.from(instanceType, 1000)); + } + + protected void testCreateWithKeyCollision(FateStore store, ServerContext sctx) { + KeyExtent ke1 = new KeyExtent(TableId.of("tableId1"), new Text("zzz"), new Text("aaa")); + KeyExtent ke2 = new KeyExtent(TableId.of("tableId2"), new Text("ddd"), new Text("bbb")); + + FateKey fateKey1 = FateKey.forSplit(ke1); + FateKey fateKey2 = FateKey.forSplit(ke2); + FateId fateId1 = store.create(fateKey1); + + FateTxStore txStore = store.reserve(fateId1); + try { + try { + store.create(fateKey2); + fail("Expected IllegalStateException due to hashing collision"); + } catch (Exception e) { + assertInstanceOf(IllegalStateException.class, e); + assertEquals("Collision detected for tid 1000", e.getMessage()); + } + } finally { + txStore.delete(); + txStore.unreserve(0, TimeUnit.SECONDS); + } + } private static class TestOperation2 extends TestRepo { diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java index bbb6aa1f6b5..f990ec8e1e1 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java @@ -27,6 +27,7 @@ import java.util.UUID; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.server.ServerContext; @@ -61,12 +62,14 @@ public static void teardown() throws Exception { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred, + FateIdGenerator fateIdGenerator) throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute(new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred), sctx); + testMethod.execute(new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred, fateIdGenerator), + sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java index 18de2ef9221..c8688a89519 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZookeeperFateIT.java @@ -29,6 +29,7 @@ import java.util.UUID; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.fate.ZooStore; @@ -67,13 +68,15 @@ public static void teardown() throws Exception { } @Override - public void executeTest(FateTestExecutor testMethod, int maxDeferred) throws Exception { + public void executeTest(FateTestExecutor testMethod, int maxDeferred, + FateIdGenerator fateIdGenerator) throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute(new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred), sctx); + testMethod.execute(new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred, fateIdGenerator), + sctx); } @Override From 9a73cf6008460fc18bdea2f92ee0586f9031a875 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 06:31:15 -0500 Subject: [PATCH 07/16] Update FateLogger Co-authored-by: Keith Turner --- .../main/java/org/apache/accumulo/core/logging/FateLogger.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bdca78ae0c9..0a103e39000 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 @@ -147,7 +147,7 @@ public boolean isDeferredOverflow() { public FateId create(FateKey fateKey) { FateId fateId = store.create(fateKey); if (storeLog.isTraceEnabled()) { - storeLog.trace("{} created fate transaction", fateId); + storeLog.trace("{} created fate transaction using key : {}", fateId, fateKey); } return fateId; } From 27107b9e0d94079a456e57e08b5a65b1bfbbe1d9 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 06:39:42 -0500 Subject: [PATCH 08/16] test improvements --- .../accumulo/test/fate/accumulo/FateStoreIT.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index df3a4aac143..0baf2f92deb 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -26,7 +26,6 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; import java.time.Duration; import java.util.HashSet; @@ -316,6 +315,7 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte // We have an existing transaction with the same key in progress // so should not be allowed assertThrows(IllegalStateException.class, () -> store.create(fateKey)); + assertEquals(TStatus.IN_PROGRESS, txStore.getStatus()); } finally { txStore.delete(); txStore.unreserve(0, TimeUnit.SECONDS); @@ -352,13 +352,9 @@ protected void testCreateWithKeyCollision(FateStore store, ServerContex FateTxStore txStore = store.reserve(fateId1); try { - try { - store.create(fateKey2); - fail("Expected IllegalStateException due to hashing collision"); - } catch (Exception e) { - assertInstanceOf(IllegalStateException.class, e); - assertEquals("Collision detected for tid 1000", e.getMessage()); - } + var e = assertThrows(IllegalStateException.class, () -> store.create(fateKey2)); + assertEquals("Collision detected for tid 1000", e.getMessage()); + assertEquals(fateKey1, txStore.getKey().orElseThrow()); } finally { txStore.delete(); txStore.unreserve(0, TimeUnit.SECONDS); From 7793319f853224b0f79ae47dd6acac84bff3d8e5 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 06:51:53 -0500 Subject: [PATCH 09/16] fix race condition in create(fateKey) --- .../accumulo/core/fate/AbstractFateStore.java | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index bba31c529f5..60c456df59d 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -257,24 +257,26 @@ public int getDeferredCount() { @Override public FateId create(FateKey fateKey) { FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); - Pair> statusAndKey = getStatusAndKey(fateId); - TStatus status = statusAndKey.getFirst(); - Optional tFateKey = statusAndKey.getSecond(); - // Case 1: Status of UNKNOWN means doesn't exist, so we can create - if (status == TStatus.UNKNOWN) { + try { create(fateId, fateKey); - // Case 2: Status is NEW so this is unseeded, we can return and allow the calling code + } catch (IllegalStateException e) { + Pair> statusAndKey = getStatusAndKey(fateId); + TStatus status = statusAndKey.getFirst(); + Optional tFateKey = statusAndKey.getSecond(); + + // Case 1: Status is NEW so this is unseeded, we can return and allow the calling code // to reserve/seed as long as the existing key is the same and not different as that would // mean a collision - } else if (status == TStatus.NEW) { - Preconditions.checkState(tFateKey.isPresent(), "Tx key column is missing"); - Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), - "Collision detected for tid %s", fateId.getTid()); - // Case 3: Status is some other state which means already in progress - } else { - throw new IllegalStateException( - "Existing transaction already exists for: " + fateId.getTid()); + if (status == TStatus.NEW) { + Preconditions.checkState(tFateKey.isPresent(), "Tx key column is missing"); + Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), + "Collision detected for tid %s", fateId.getTid()); + // Case 2: Status is some other state which means already in progress + } else { + throw new IllegalStateException( + "Existing transaction already exists for: " + fateId.getTid()); + } } return fateId; From 8aad33ea979b799bbc41b10539e6cbcca37b82d3 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 08:14:10 -0500 Subject: [PATCH 10/16] Add a test for a fate key collision with random fate id --- .../core/fate/accumulo/AccumuloStore.java | 6 ++- .../core/fate/accumulo/FateMutatorImpl.java | 3 +- .../fate/accumulo/AccumuloStoreFateIT.java | 22 ++++++++- .../test/fate/accumulo/FateStoreIT.java | 48 +++++++++++++++++-- .../test/fate/zookeeper/ZooStoreFateIT.java | 40 ++++++++++++++++ 5 files changed, 111 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index a70e1d97617..810e9815b49 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -211,7 +211,11 @@ protected FateInstanceType getInstanceType() { } static Range getRow(FateId fateId) { - return new Range("tx_" + fateId.getHexTid()); + return new Range(getRowId(fateId)); + } + + public static String getRowId(FateId fateId) { + return "tx_" + fateId.getHexTid(); } private FateMutatorImpl newMutator(FateId fateId) { diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java index 3cc31bb6fcd..342bfc6e83c 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/FateMutatorImpl.java @@ -20,6 +20,7 @@ import static org.apache.accumulo.core.fate.AbstractFateStore.serialize; import static org.apache.accumulo.core.fate.accumulo.AccumuloStore.getRow; +import static org.apache.accumulo.core.fate.accumulo.AccumuloStore.getRowId; import static org.apache.accumulo.core.fate.accumulo.AccumuloStore.invertRepo; import java.util.Objects; @@ -57,7 +58,7 @@ public FateMutatorImpl(ClientContext context, String tableName, FateId fateId) { this.context = Objects.requireNonNull(context); this.tableName = Objects.requireNonNull(tableName); this.fateId = fateId; - this.mutation = new ConditionalMutation(new Text("tx_" + fateId.getHexTid())); + this.mutation = new ConditionalMutation(new Text(getRowId(fateId))); } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java index 7e2f0d8dffe..da0b62e14d8 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/AccumuloStoreFateIT.java @@ -18,11 +18,20 @@ */ package org.apache.accumulo.test.fate.accumulo; +import static org.apache.accumulo.core.fate.accumulo.AccumuloStore.getRowId; + import org.apache.accumulo.core.client.Accumulo; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.MutationsRejectedException; +import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; +import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.accumulo.AccumuloStore; +import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxColumnFamily; import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.apache.accumulo.server.ServerContext; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -41,7 +50,7 @@ public static void teardown() { @Override public void executeTest(FateTestExecutor testMethod, int maxDeferred, FateIdGenerator fateIdGenerator) throws Exception { - String table = getUniqueNames(1)[0]; + String table = getUniqueNames(1)[0] + "fatestore"; try (ClientContext client = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { client.tableOperations().create(table); @@ -50,4 +59,15 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, } } + @Override + protected void deleteKey(FateId fateId, ServerContext context) { + String table = getUniqueNames(1)[0] + "fatestore"; + try (BatchWriter bw = context.createBatchWriter(table)) { + Mutation mut = new Mutation(getRowId(fateId)); + TxColumnFamily.TX_KEY_COLUMN.putDelete(mut); + bw.addMutation(mut); + } catch (TableNotFoundException | MutationsRejectedException e) { + throw new IllegalStateException(e); + } + } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index 0baf2f92deb..ce5adba7b1c 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -239,7 +239,8 @@ public void testCreateWithKey() throws Exception { } protected void testCreateWithKey(FateStore store, ServerContext sctx) { - KeyExtent ke1 = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); + KeyExtent ke1 = + new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); FateKey fateKey1 = FateKey.forSplit(ke1); FateId fateId1 = store.create(fateKey1); @@ -275,7 +276,8 @@ public void testCreateWithKeyDuplicate() throws Exception { } protected void testCreateWithKeyDuplicate(FateStore store, ServerContext sctx) { - KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); + KeyExtent ke = + new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); // Creating with the same key should be fine if the status is NEW // It should just return the same id and allow us to continue reserving @@ -302,7 +304,8 @@ public void testCreateWithKeyInProgress() throws Exception { } protected void testCreateWithKeyInProgress(FateStore store, ServerContext sctx) { - KeyExtent ke = new KeyExtent(TableId.of("tableId"), new Text("zzz"), new Text("aaa")); + KeyExtent ke = + new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); FateKey fateKey = FateKey.forSplit(ke); FateId fateId = store.create(fateKey); @@ -343,8 +346,9 @@ public void testCreateWithKeyCollision() throws Exception { } protected void testCreateWithKeyCollision(FateStore store, ServerContext sctx) { - KeyExtent ke1 = new KeyExtent(TableId.of("tableId1"), new Text("zzz"), new Text("aaa")); - KeyExtent ke2 = new KeyExtent(TableId.of("tableId2"), new Text("ddd"), new Text("bbb")); + String[] tables = getUniqueNames(2); + KeyExtent ke1 = new KeyExtent(TableId.of(tables[0]), new Text("zzz"), new Text("aaa")); + KeyExtent ke2 = new KeyExtent(TableId.of(tables[1]), new Text("ddd"), new Text("bbb")); FateKey fateKey1 = FateKey.forSplit(ke1); FateKey fateKey2 = FateKey.forSplit(ke2); @@ -362,6 +366,40 @@ protected void testCreateWithKeyCollision(FateStore store, ServerContex } + @Test + public void testCollisionWithRandomFateId() throws Exception { + executeTest(this::testCollisionWithRandomFateId); + } + + protected void testCollisionWithRandomFateId(FateStore store, ServerContext sctx) { + KeyExtent ke = + new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); + + FateKey fateKey = FateKey.forSplit(ke); + FateId fateId = store.create(fateKey); + + // After create a fate transaction using a key we can simulate a collision with + // a random FateId by deleting the key out of Fate and calling create again to verify + // it detects the key is missing. Then we can continue and see if we can still reserve + // and use the existing transaction, which we should. + deleteKey(fateId, sctx); + var e = assertThrows(IllegalStateException.class, () -> store.create(fateKey)); + assertEquals("Tx key column is missing", e.getMessage()); + + // We should still be able to reserve and continue when not using a key + FateTxStore txStore = store.reserve(fateId); + try { + assertTrue(txStore.timeCreated() > 0); + assertEquals(TStatus.NEW, txStore.getStatus()); + } finally { + txStore.delete(); + txStore.unreserve(0, TimeUnit.SECONDS); + } + + } + + protected abstract void deleteKey(FateId fateId, ServerContext sctx); + private static class TestOperation2 extends TestRepo { private static final long serialVersionUID = 1L; diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java index f990ec8e1e1..c679e0d54f5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooStoreFateIT.java @@ -24,12 +24,18 @@ import static org.easymock.EasyMock.replay; import java.io.File; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.util.UUID; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore.FateIdGenerator; +import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; 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.server.ServerContext; import org.apache.accumulo.test.fate.accumulo.FateStoreIT; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; @@ -72,4 +78,38 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, testMethod.execute(new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk, maxDeferred, fateIdGenerator), sctx); } + + @Override + protected void deleteKey(FateId fateId, ServerContext sctx) { + try { + // We have to use reflection since the NodeValue is internal to the store + + // Grab both the constructor that uses the serialized bytes and status + Class nodeClass = Class.forName(ZooStore.class.getName() + "$NodeValue"); + Constructor statusCons = nodeClass.getDeclaredConstructor(TStatus.class); + Constructor serializedCons = nodeClass.getDeclaredConstructor(byte[].class); + statusCons.setAccessible(true); + serializedCons.setAccessible(true); + + // Get the status field so it can be read and the serialize method + Field nodeStatus = nodeClass.getDeclaredField("status"); + Method nodeSerialize = nodeClass.getDeclaredMethod("serialize"); + nodeStatus.setAccessible(true); + nodeSerialize.setAccessible(true); + + // Get the existing status for the node and build a new node with an empty key + // but uses the existing tid + String txPath = ZK_ROOT + Constants.ZFATE + "/tx_" + fateId.getHexTid(); + Object currentNode = serializedCons.newInstance(new Object[] {zk.getData(txPath)}); + TStatus currentStatus = (TStatus) nodeStatus.get(currentNode); + // replace the node with no key and just a tid and existing status + Object newNode = statusCons.newInstance(currentStatus); + + // Replace the transaction with the same status and no key + zk.putPersistentData(txPath, (byte[]) nodeSerialize.invoke(newNode), + NodeExistsPolicy.OVERWRITE); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } } From 9fc782f44c2b08a0f13de2508abd64ad2a245b74 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 08:22:15 -0500 Subject: [PATCH 11/16] error message and test updates --- .../org/apache/accumulo/core/fate/AbstractFateStore.java | 3 ++- .../org/apache/accumulo/test/fate/accumulo/FateStoreIT.java | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 60c456df59d..44db501c33c 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -269,7 +269,8 @@ public FateId create(FateKey fateKey) { // to reserve/seed as long as the existing key is the same and not different as that would // mean a collision if (status == TStatus.NEW) { - Preconditions.checkState(tFateKey.isPresent(), "Tx key column is missing"); + Preconditions.checkState(tFateKey.isPresent(), "Tx Key is missing from tid %s", + fateId.getTid()); Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), "Collision detected for tid %s", fateId.getTid()); // Case 2: Status is some other state which means already in progress diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index ce5adba7b1c..f8688636ed7 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -242,6 +242,7 @@ protected void testCreateWithKey(FateStore store, ServerContext sctx) { KeyExtent ke1 = new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); + long existing = store.list().count(); FateKey fateKey1 = FateKey.forSplit(ke1); FateId fateId1 = store.create(fateKey1); @@ -261,7 +262,7 @@ protected void testCreateWithKey(FateStore store, ServerContext sctx) { assertEquals(TStatus.NEW, txStore2.getStatus()); assertEquals(fateKey2, txStore2.getKey().orElseThrow()); - assertEquals(2, store.list().count()); + assertEquals(existing + 2, store.list().count()); } finally { txStore1.delete(); txStore2.delete(); @@ -384,7 +385,7 @@ protected void testCollisionWithRandomFateId(FateStore store, ServerCon // and use the existing transaction, which we should. deleteKey(fateId, sctx); var e = assertThrows(IllegalStateException.class, () -> store.create(fateKey)); - assertEquals("Tx key column is missing", e.getMessage()); + assertEquals("Tx Key is missing from tid " + fateId.getTid(), e.getMessage()); // We should still be able to reserve and continue when not using a key FateTxStore txStore = store.reserve(fateId); From 3b2ca7776def8871f28abf35ec660ce8684ab7f8 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 08:34:27 -0500 Subject: [PATCH 12/16] Update create(fateId,fateKey) to handle retry --- .../core/fate/accumulo/AccumuloStore.java | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index 810e9815b49..328560b1506 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -43,7 +43,6 @@ 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.accumulo.FateMutator.Status; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.RepoColumnFamily; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxColumnFamily; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxInfoColumnFamily; @@ -123,12 +122,34 @@ public FateId getFateId() { @Override protected void create(FateId fateId, FateKey fateKey) { - var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) - .putCreateTime(System.currentTimeMillis()).tryMutate(); + final int maxAttempts = 5; + + for (int attempt = 0; attempt < maxAttempts; attempt++) { + + if (attempt >= 1) { + log.debug("Failed to create transaction with fateId {} and fateKey {}, trying again", + fateId, fateKey); + UtilWaitThread.sleep(100); + } + + var status = newMutator(fateId).requireStatus().putStatus(TStatus.NEW).putKey(fateKey) + .putCreateTime(System.currentTimeMillis()).tryMutate(); + + switch (status) { + case ACCEPTED: + return; + case UNKNOWN: + continue; + case REJECTED: + throw new IllegalStateException("Attempt to create transaction with fateId " + fateId + + " and fateKey " + fateKey + " was rejected"); + default: + throw new IllegalStateException("Unknown status " + status); + } + } - // TODO: Any reason to retry here? - Preconditions.checkState(status == Status.ACCEPTED, - "Failed to create transaction with fateId %s and fateKey %s", fateId, fateKey); + throw new IllegalStateException("Failed to create transaction with fateId " + fateId + + " and fateKey " + fateKey + " after " + maxAttempts + " attempts"); } @Override From 4f6673699fe5c81e19b41f1d943b5012b2e24b88 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 11:57:58 -0500 Subject: [PATCH 13/16] Replace store.create(key) with store.createAndReserve(key) this will make sure that if a transaction is created by a key it will reserve at the same time to prevent race conditions --- .../accumulo/core/fate/AbstractFateStore.java | 40 ++++++++++- .../apache/accumulo/core/fate/FateStore.java | 15 +++- .../accumulo/core/logging/FateLogger.java | 15 ++-- .../apache/accumulo/core/fate/TestStore.java | 22 ++++-- .../test/fate/accumulo/FateStoreIT.java | 71 +++++++++++++------ 5 files changed, 129 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 44db501c33c..13d63da1cb9 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -254,8 +254,7 @@ public int getDeferredCount() { } } - @Override - public FateId create(FateKey fateKey) { + private FateId create(FateKey fateKey) { FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); try { @@ -283,6 +282,43 @@ public FateId create(FateKey fateKey) { return fateId; } + @Override + public Optional> createAndReserve(FateKey fateKey) { + FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); + + final Optional> txStore; + + // First make sure we can reserve in memory the fateId, if not + // we can return an empty Optional as it is reserved and in progress + // This reverses the usual order of creation and then reservation but + // this prevents a race condition by ensuring we can reserve first. + // This will create the FateTxStore before creation but this object + // is not exposed until after creation is finished so there should not + // be any errors. + synchronized (this) { + txStore = tryReserve(fateId); + } + + if (txStore.isPresent()) { + try { + Preconditions.checkState(create(fateKey) != null, + "Unexpected null FateId when creating and reserving fateKey %s", fateKey); + } catch (Exception e) { + // Clean up the reservation if the creation failed + synchronized (this) { + reserved.remove(fateId); + } + if (e instanceof IllegalStateException) { + throw e; + } else { + throw new IllegalStateException(e); + } + } + } + + return txStore; + } + protected abstract void create(FateId fateId, FateKey fateKey); protected abstract Pair> getStatusAndKey(FateId fateId); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/FateStore.java index 7efee2267fa..088e5025229 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 @@ -39,7 +39,20 @@ public interface FateStore extends ReadOnlyFateStore { */ FateId create(); - FateId create(FateKey fateKey); + /** + * Creates and reserves a transaction using the given key. If something is already running for the + * given key, then Optional.empty() will be returned. When this returns a non-empty id, it will be + * in the new state. + * + *

+ * In the case where a process dies in the middle of a call to this. If later, another call is + * made with the same key and its in the new state then the FateId for that key will be returned. + *

+ * + * @throws IllegalStateException when there is an unexpected collision. This can occur if two key + * hash to the same FateId or if a random FateId already exists. + */ + Optional> createAndReserve(FateKey fateKey); /** * An interface that allows read/write access to the data related to a single fate operation. 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 0a103e39000..0879fbaea85 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 @@ -144,12 +144,19 @@ public boolean isDeferredOverflow() { } @Override - public FateId create(FateKey fateKey) { - FateId fateId = store.create(fateKey); + public Optional> createAndReserve(FateKey fateKey) { + Optional> txStore = store.createAndReserve(fateKey); if (storeLog.isTraceEnabled()) { - storeLog.trace("{} created fate transaction using key : {}", fateId, fateKey); + if (txStore.isPresent()) { + storeLog.trace("{} created and reserved fate transaction using key : {}", + txStore.orElseThrow().getID(), fateKey); + } else { + storeLog.trace( + "fate transaction was not created using key : {}, existing transaction exists", + fateKey); + } } - return fateId; + return txStore; } }; } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 627b60e3ccd..5ba0c214c09 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -56,14 +56,28 @@ public FateId create() { } @Override - public FateId create(FateKey key) { + public Optional> createAndReserve(FateKey key) { HashCode hashCode = Hashing.murmur3_128().hashBytes(key.getSerialized()); long tid = hashCode.asLong() & 0x7fffffffffffffffL; FateId fateId = FateId.from(fateInstanceType, tid); - if (statuses.putIfAbsent(fateId, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { - throw new IllegalStateException("Transaction with fateId " + fateId + " already exists"); + + Optional> txStore = tryReserve(fateId); + if (txStore.isPresent()) { + try { + if (statuses.putIfAbsent(fateId, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { + throw new IllegalStateException("Transaction with fateId " + fateId + " already exists"); + } + } catch (Exception e) { + reserved.remove(fateId); + if (e instanceof IllegalStateException) { + throw e; + } else { + throw new IllegalStateException(e); + } + } } - return fateId; + + return txStore; } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index f8688636ed7..b7bd92c94d9 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -27,6 +27,8 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.time.Duration; import java.util.HashSet; import java.util.List; @@ -58,8 +60,22 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; +import com.google.common.base.Throwables; + public abstract class FateStoreIT extends SharedMiniClusterBase implements FateTestRunner { + private static final Method fsCreateByKeyMethod; + + static { + try { + // Private method, need to capture for testing + fsCreateByKeyMethod = AbstractFateStore.class.getDeclaredMethod("create", FateKey.class); + fsCreateByKeyMethod.setAccessible(true); + } catch (NoSuchMethodException e) { + throw new RuntimeException(e); + } + } + @Override protected Duration defaultTimeout() { return Duration.ofMinutes(1); @@ -244,15 +260,14 @@ protected void testCreateWithKey(FateStore store, ServerContext sctx) { long existing = store.list().count(); FateKey fateKey1 = FateKey.forSplit(ke1); - FateId fateId1 = store.create(fateKey1); - FateKey fateKey2 = FateKey.forCompactionCommit(ExternalCompactionId.generate(UUID.randomUUID())); - FateId fateId2 = store.create(fateKey2); - assertNotEquals(fateId1, fateId2); - FateTxStore txStore1 = store.reserve(fateId1); - FateTxStore txStore2 = store.reserve(fateId2); + FateTxStore txStore1 = store.createAndReserve(fateKey1).orElseThrow(); + FateTxStore txStore2 = store.createAndReserve(fateKey2).orElseThrow(); + + assertNotEquals(txStore1.getID(), txStore2.getID()); + try { assertTrue(txStore1.timeCreated() > 0); assertEquals(TStatus.NEW, txStore1.getStatus()); @@ -281,13 +296,14 @@ protected void testCreateWithKeyDuplicate(FateStore store, ServerContex new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); // Creating with the same key should be fine if the status is NEW - // It should just return the same id and allow us to continue reserving + // A second call to createAndReserve() should just return an empty optional + // since it's already in reserved and in progress FateKey fateKey = FateKey.forSplit(ke); - FateId fateId1 = store.create(fateKey); - FateId fateId2 = store.create(fateKey); - assertEquals(fateId1, fateId2); + FateTxStore txStore = store.createAndReserve(fateKey).orElseThrow(); + + // second call is empty + assertTrue(store.createAndReserve(fateKey).isEmpty()); - FateTxStore txStore = store.reserve(fateId1); try { assertTrue(txStore.timeCreated() > 0); assertEquals(TStatus.NEW, txStore.getStatus()); @@ -307,18 +323,17 @@ public void testCreateWithKeyInProgress() throws Exception { protected void testCreateWithKeyInProgress(FateStore store, ServerContext sctx) { KeyExtent ke = new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); - FateKey fateKey = FateKey.forSplit(ke); - FateId fateId = store.create(fateKey); - FateTxStore txStore = store.reserve(fateId); + FateTxStore txStore = store.createAndReserve(fateKey).orElseThrow(); + ; try { assertTrue(txStore.timeCreated() > 0); txStore.setStatus(TStatus.IN_PROGRESS); // We have an existing transaction with the same key in progress // so should not be allowed - assertThrows(IllegalStateException.class, () -> store.create(fateKey)); + assertThrows(IllegalStateException.class, () -> create(store, fateKey)); assertEquals(TStatus.IN_PROGRESS, txStore.getStatus()); } finally { txStore.delete(); @@ -327,8 +342,7 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte try { // After deletion, make sure we can create again with the same key - fateId = store.create(fateKey); - txStore = store.reserve(fateId); + txStore = store.createAndReserve(fateKey).orElseThrow(); assertTrue(txStore.timeCreated() > 0); assertEquals(TStatus.NEW, txStore.getStatus()); } finally { @@ -353,11 +367,10 @@ protected void testCreateWithKeyCollision(FateStore store, ServerContex FateKey fateKey1 = FateKey.forSplit(ke1); FateKey fateKey2 = FateKey.forSplit(ke2); - FateId fateId1 = store.create(fateKey1); - FateTxStore txStore = store.reserve(fateId1); + FateTxStore txStore = store.createAndReserve(fateKey1).orElseThrow(); try { - var e = assertThrows(IllegalStateException.class, () -> store.create(fateKey2)); + var e = assertThrows(IllegalStateException.class, () -> create(store, fateKey2)); assertEquals("Collision detected for tid 1000", e.getMessage()); assertEquals(fateKey1, txStore.getKey().orElseThrow()); } finally { @@ -372,22 +385,24 @@ public void testCollisionWithRandomFateId() throws Exception { executeTest(this::testCollisionWithRandomFateId); } - protected void testCollisionWithRandomFateId(FateStore store, ServerContext sctx) { + protected void testCollisionWithRandomFateId(FateStore store, ServerContext sctx) + throws Exception { KeyExtent ke = new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); FateKey fateKey = FateKey.forSplit(ke); - FateId fateId = store.create(fateKey); + FateId fateId = create(store, fateKey); // After create a fate transaction using a key we can simulate a collision with // a random FateId by deleting the key out of Fate and calling create again to verify // it detects the key is missing. Then we can continue and see if we can still reserve // and use the existing transaction, which we should. deleteKey(fateId, sctx); - var e = assertThrows(IllegalStateException.class, () -> store.create(fateKey)); + var e = assertThrows(IllegalStateException.class, () -> store.createAndReserve(fateKey)); assertEquals("Tx Key is missing from tid " + fateId.getTid(), e.getMessage()); // We should still be able to reserve and continue when not using a key + // just like a normal transaction FateTxStore txStore = store.reserve(fateId); try { assertTrue(txStore.timeCreated() > 0); @@ -399,6 +414,16 @@ protected void testCollisionWithRandomFateId(FateStore store, ServerCon } + // create(fateKey) method is private so expose for testing to check error states + protected FateId create(FateStore store, FateKey fateKey) throws Exception { + try { + return (FateId) fsCreateByKeyMethod.invoke(store, fateKey); + } catch (InvocationTargetException e) { + Exception rootCause = (Exception) Throwables.getRootCause(e); + throw rootCause; + } + } + protected abstract void deleteKey(FateId fateId, ServerContext sctx); private static class TestOperation2 extends TestRepo { From 2ad01d328f88e72e0ba508292a084fa1c599ddd1 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 13:53:00 -0500 Subject: [PATCH 14/16] Return empty optional for fate store instead of throwing error if status is not new --- .../accumulo/core/fate/AbstractFateStore.java | 34 ++++++++++++++----- .../test/fate/accumulo/FateStoreIT.java | 18 +++++----- 2 files changed, 35 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 13d63da1cb9..8b803ee9b39 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -254,7 +254,7 @@ public int getDeferredCount() { } } - private FateId create(FateKey fateKey) { + private Optional create(FateKey fateKey) { FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); try { @@ -273,19 +273,20 @@ private FateId create(FateKey fateKey) { Preconditions.checkState(fateKey.equals(tFateKey.orElseThrow()), "Collision detected for tid %s", fateId.getTid()); // Case 2: Status is some other state which means already in progress + // so we can just log and return empty optional } else { - throw new IllegalStateException( - "Existing transaction already exists for: " + fateId.getTid()); + log.trace("Existing transaction {} already exists for key {} with status {}", fateId, + fateKey, status); + return Optional.empty(); } } - return fateId; + return Optional.of(fateId); } @Override public Optional> createAndReserve(FateKey fateKey) { FateId fateId = fateIdGenerator.fromTypeAndKey(getInstanceType(), fateKey); - final Optional> txStore; // First make sure we can reserve in memory the fateId, if not @@ -295,16 +296,28 @@ public Optional> createAndReserve(FateKey fateKey) { // This will create the FateTxStore before creation but this object // is not exposed until after creation is finished so there should not // be any errors. + final Optional> reservedTxStore; synchronized (this) { - txStore = tryReserve(fateId); + reservedTxStore = tryReserve(fateId); } - if (txStore.isPresent()) { + // If present we were able to reserve so try and create + if (reservedTxStore.isPresent()) { try { - Preconditions.checkState(create(fateKey) != null, - "Unexpected null FateId when creating and reserving fateKey %s", fateKey); + if (create(fateKey).isPresent()) { + txStore = reservedTxStore; + } else { + // We already exist in a non-new state then un-reserve and an empty + // Optional will be returned. This is expected to happen when the + // system is busy and operations are not running, and we keep seeding them + synchronized (this) { + reserved.remove(fateId); + } + txStore = Optional.empty(); + } } catch (Exception e) { // Clean up the reservation if the creation failed + // And then throw error synchronized (this) { reserved.remove(fateId); } @@ -314,6 +327,9 @@ public Optional> createAndReserve(FateKey fateKey) { throw new IllegalStateException(e); } } + } else { + // Could not reserve so return empty + txStore = Optional.empty(); } return txStore; diff --git a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java index b7bd92c94d9..181a21b9c64 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/accumulo/FateStoreIT.java @@ -27,11 +27,11 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.time.Duration; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.Executors; @@ -320,7 +320,8 @@ public void testCreateWithKeyInProgress() throws Exception { executeTest(this::testCreateWithKeyInProgress); } - protected void testCreateWithKeyInProgress(FateStore store, ServerContext sctx) { + protected void testCreateWithKeyInProgress(FateStore store, ServerContext sctx) + throws Exception { KeyExtent ke = new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); FateKey fateKey = FateKey.forSplit(ke); @@ -332,8 +333,8 @@ protected void testCreateWithKeyInProgress(FateStore store, ServerConte txStore.setStatus(TStatus.IN_PROGRESS); // We have an existing transaction with the same key in progress - // so should not be allowed - assertThrows(IllegalStateException.class, () -> create(store, fateKey)); + // so should return an empty Optional + assertTrue(create(store, fateKey).isEmpty()); assertEquals(TStatus.IN_PROGRESS, txStore.getStatus()); } finally { txStore.delete(); @@ -391,7 +392,7 @@ protected void testCollisionWithRandomFateId(FateStore store, ServerCon new KeyExtent(TableId.of(getUniqueNames(1)[0]), new Text("zzz"), new Text("aaa")); FateKey fateKey = FateKey.forSplit(ke); - FateId fateId = create(store, fateKey); + FateId fateId = create(store, fateKey).orElseThrow(); // After create a fate transaction using a key we can simulate a collision with // a random FateId by deleting the key out of Fate and calling create again to verify @@ -415,10 +416,11 @@ protected void testCollisionWithRandomFateId(FateStore store, ServerCon } // create(fateKey) method is private so expose for testing to check error states - protected FateId create(FateStore store, FateKey fateKey) throws Exception { + @SuppressWarnings("unchecked") + protected Optional create(FateStore store, FateKey fateKey) throws Exception { try { - return (FateId) fsCreateByKeyMethod.invoke(store, fateKey); - } catch (InvocationTargetException e) { + return (Optional) fsCreateByKeyMethod.invoke(store, fateKey); + } catch (Exception e) { Exception rootCause = (Exception) Throwables.getRootCause(e); throw rootCause; } From ead1ccc79f2eca4abc7e1531446e5a47d949fad5 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 15:18:37 -0500 Subject: [PATCH 15/16] Updates based on comments --- .../accumulo/core/fate/AbstractFateStore.java | 4 +++ .../apache/accumulo/core/fate/TestStore.java | 29 ++++--------------- 2 files changed, 9 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 8b803ee9b39..048c15e12ec 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -304,7 +304,10 @@ public Optional> createAndReserve(FateKey fateKey) { // If present we were able to reserve so try and create if (reservedTxStore.isPresent()) { try { + var fateIdFromCreate = create(fateKey); if (create(fateKey).isPresent()) { + Preconditions.checkState(fateId.equals(fateIdFromCreate.orElseThrow()), + "Transaction creation returned unexpected %s, expected %s", fateIdFromCreate, fateId); txStore = reservedTxStore; } else { // We already exist in a non-new state then un-reserve and an empty @@ -329,6 +332,7 @@ public Optional> createAndReserve(FateKey fateKey) { } } else { // Could not reserve so return empty + log.trace("Another thread currently has transaction {} key {} reserved", fateId, fateKey); txStore = Optional.empty(); } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java index 5ba0c214c09..18089848df1 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/TestStore.java @@ -32,11 +32,12 @@ import java.util.function.Consumer; import java.util.stream.Stream; +import org.apache.accumulo.core.fate.FateStore.FateTxStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.FateIdStatus; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.ReadOnlyFateTxStore; +import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.util.Pair; -import com.google.common.hash.HashCode; -import com.google.common.hash.Hashing; - /** * Transient in memory store for transactions. */ @@ -57,27 +58,7 @@ public FateId create() { @Override public Optional> createAndReserve(FateKey key) { - HashCode hashCode = Hashing.murmur3_128().hashBytes(key.getSerialized()); - long tid = hashCode.asLong() & 0x7fffffffffffffffL; - FateId fateId = FateId.from(fateInstanceType, tid); - - Optional> txStore = tryReserve(fateId); - if (txStore.isPresent()) { - try { - if (statuses.putIfAbsent(fateId, new Pair<>(TStatus.NEW, Optional.of(key))) != null) { - throw new IllegalStateException("Transaction with fateId " + fateId + " already exists"); - } - } catch (Exception e) { - reserved.remove(fateId); - if (e instanceof IllegalStateException) { - throw e; - } else { - throw new IllegalStateException(e); - } - } - } - - return txStore; + throw new UnsupportedOperationException(); } @Override From 9dfa79cba9d0071f0d2bf87708e90e1e749fac8e Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Fri, 9 Feb 2024 15:22:25 -0500 Subject: [PATCH 16/16] Don't call create twice --- .../java/org/apache/accumulo/core/fate/AbstractFateStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java index 048c15e12ec..702313f5abf 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AbstractFateStore.java @@ -305,7 +305,7 @@ public Optional> createAndReserve(FateKey fateKey) { if (reservedTxStore.isPresent()) { try { var fateIdFromCreate = create(fateKey); - if (create(fateKey).isPresent()) { + if (fateIdFromCreate.isPresent()) { Preconditions.checkState(fateId.equals(fateIdFromCreate.orElseThrow()), "Transaction creation returned unexpected %s, expected %s", fateIdFromCreate, fateId); txStore = reservedTxStore;