From 9bd63209eea33504b127d066a9864b2c7fdd10dc Mon Sep 17 00:00:00 2001 From: Gary Helmling Date: Mon, 4 May 2015 19:43:26 -0700 Subject: [PATCH 1/5] TEPHRA-96 Transaction checkpoints: support multiple write pointers per tx --- .../main/java/co/cask/tephra/Transaction.java | 124 +- .../tephra/AbstractTransactionAwareTable.java | 23 +- .../co/cask/tephra/TransactionContext.java | 61 +- .../co/cask/tephra/TransactionManager.java | 124 +- .../cask/tephra/TransactionSystemClient.java | 8 + .../main/java/co/cask/tephra/TxConstants.java | 9 +- .../TransactionConverterUtils.java | 8 +- .../distributed/TransactionServiceClient.java | 18 + .../TransactionServiceThriftClient.java | 4 + .../TransactionServiceThriftHandler.java | 34 +- .../tephra/distributed/thrift/TBoolean.java | 24 +- .../thrift/TInvalidTruncateTimeException.java | 24 +- .../distributed/thrift/TTransaction.java | 328 +++++- ...nsactionCouldNotTakeSnapshotException.java | 24 +- .../TTransactionNotInProgressException.java | 24 +- .../thrift/TTransactionServer.java | 1000 ++++++++++++++++- .../distributed/thrift/TTransactionType.java | 4 + .../inmemory/DetachedTxSystemClient.java | 22 +- .../inmemory/InMemoryTxSystemClient.java | 5 + .../inmemory/MinimalTxSystemClient.java | 6 + .../cask/tephra/persist/TransactionEdit.java | 109 +- .../cask/tephra/snapshot/SnapshotCodecV2.java | 4 +- .../cask/tephra/snapshot/SnapshotCodecV4.java | 98 ++ .../java/co/cask/tephra/util/TxUtils.java | 3 +- .../src/main/thrift/transaction.thrift | 3 + .../AbstractTransactionStateStorageTest.java | 6 +- .../LocalTransactionStateStorageTest.java | 2 +- .../hbase96/TransactionAwareHTable.java | 64 +- .../coprocessor/TransactionProcessor.java | 21 +- .../TransactionVisibilityFilter.java | 24 +- .../hbase96/TransactionAwareHTableTest.java | 169 +++ .../hbase98/TransactionAwareHTable.java | 63 +- .../coprocessor/TransactionProcessor.java | 21 +- .../TransactionVisibilityFilter.java | 24 +- .../hbase98/TransactionAwareHTableTest.java | 164 +++ 35 files changed, 2397 insertions(+), 252 deletions(-) create mode 100644 tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV4.java diff --git a/tephra-api/src/main/java/co/cask/tephra/Transaction.java b/tephra-api/src/main/java/co/cask/tephra/Transaction.java index e64cde55..617b538f 100644 --- a/tephra-api/src/main/java/co/cask/tephra/Transaction.java +++ b/tephra-api/src/main/java/co/cask/tephra/Transaction.java @@ -26,10 +26,12 @@ public class Transaction { private final long readPointer; private final long writePointer; + private final long currentWritePointer; private final long[] invalids; private final long[] inProgress; private final long firstShortInProgress; private final TransactionType type; + private final long[] checkpointWritePointers; private static final long[] NO_EXCLUDES = { }; public static final long NO_TX_IN_PROGRESS = Long.MAX_VALUE; @@ -37,10 +39,23 @@ public class Transaction { public static final Transaction ALL_VISIBLE_LATEST = new Transaction(Long.MAX_VALUE, Long.MAX_VALUE, NO_EXCLUDES, NO_EXCLUDES, NO_TX_IN_PROGRESS, TransactionType.SHORT); + /** + * Creates a new short transaction. + * @param readPointer read pointer for transaction + * @param writePointer write pointer for transaction. This uniquely identifies the transaction. + * @param invalids list of invalid transactions to exclude while reading + * @param inProgress list of in-progress transactions to exclude while reading + * @param firstShortInProgress earliest in-progress short transaction + */ + public Transaction(long readPointer, long writePointer, long[] invalids, long[] inProgress, + long firstShortInProgress) { + this(readPointer, writePointer, invalids, inProgress, firstShortInProgress, TransactionType.SHORT); + } + /** * Creates a new transaction. * @param readPointer read pointer for transaction - * @param writePointer write pointer for transaction + * @param writePointer write pointer for transaction. This uniquely identifies the transaction. * @param invalids list of invalid transactions to exclude while reading * @param inProgress list of in-progress transactions to exclude while reading * @param firstShortInProgress earliest in-progress short transaction @@ -48,35 +63,69 @@ public class Transaction { */ public Transaction(long readPointer, long writePointer, long[] invalids, long[] inProgress, long firstShortInProgress, TransactionType type) { + this(readPointer, writePointer, writePointer, invalids, inProgress, firstShortInProgress, type, new long[0]); + } + + /** + * Creates a new transaction. + * @param readPointer read pointer for transaction + * @param writePointer write pointer for transaction. This uniquely identifies the transaction. + * @param currentWritePointer the current pointer to be used for any writes. + * For new transactions, this will be the same as {@code writePointer}. For checkpointed + * transactions, this will be the most recent write pointer issued. + * @param invalids list of invalid transactions to exclude while reading + * @param inProgress list of in-progress transactions to exclude while reading + * @param firstShortInProgress earliest in-progress short transaction + * @param type transaction type + * @param checkpointPointers the list of writer pointers added from checkpoints on the transaction + */ + public Transaction(long readPointer, long writePointer, long currentWritePointer, long[] invalids, long[] inProgress, + long firstShortInProgress, TransactionType type, long[] checkpointPointers) { this.readPointer = readPointer; this.writePointer = writePointer; + this.currentWritePointer = currentWritePointer; this.invalids = invalids; this.inProgress = inProgress; this.firstShortInProgress = firstShortInProgress; this.type = type; + this.checkpointWritePointers = checkpointPointers; } /** - * Creates a new short transaction. - * @param readPointer read pointer for transaction - * @param writePointer write pointer for transaction - * @param invalids list of invalid transactions to exclude while reading - * @param inProgress list of in-progress transactions to exclude while reading - * @param firstShortInProgress earliest in-progress short transaction + * Creates a new transaction for a checkpoint operation, copying all members from the original transaction, + * with the updated checkpoint write pointers. + * + * @param toCopy the original transaction containing the state to copy + * @param currentWritePointer the new write pointer to use for the transaction + * @param checkpointPointers the list of write pointers added from checkpoints on the transaction */ - public Transaction(long readPointer, long writePointer, long[] invalids, long[] inProgress, - long firstShortInProgress) { - this(readPointer, writePointer, invalids, inProgress, firstShortInProgress, TransactionType.SHORT); + public Transaction(Transaction toCopy, long currentWritePointer, long[] checkpointPointers) { + this(toCopy.getReadPointer(), toCopy.getWritePointer(), currentWritePointer, toCopy.getInvalids(), + toCopy.getInProgress(), toCopy.getFirstShortInProgress(), toCopy.getType(), checkpointPointers); } public long getReadPointer() { return readPointer; } + /** + * Returns the initial write pointer assigned to the transaction. This will remain the same for the life of the + * transaction, and uniquely identifies it with the transaction service. This value should be provided + * to identify the transaction when calling any transaction lifecycle methods on the transaction service. + */ public long getWritePointer() { return writePointer; } + /** + * Returns the write pointer to be used in persisting any changes. After a checkpoint is performed, this will differ + * from {@link #getWritePointer()}. This method should always be used when setting the timestamp for writes + * in order to ensure that the correct value is used. + */ + public long getCurrentWritePointer() { + return currentWritePointer; + } + public long[] getInvalids() { return invalids; } @@ -108,18 +157,60 @@ public long getFirstShortInProgress() { return firstShortInProgress; } + /** + * Returns true if the given version corresponds to a transaction that was in-progress at the time this transaction + * started. + */ public boolean isInProgress(long version) { return Arrays.binarySearch(inProgress, version) >= 0; } + /** + * Returns true if the given version is present in one of the arrays of excluded versions (in-progress and + * invalid transactions). + */ public boolean isExcluded(long version) { return Arrays.binarySearch(inProgress, version) >= 0 || Arrays.binarySearch(invalids, version) >= 0; } + /** + * Returns true if the the given version corresponds to one of the checkpoint versions in the current + * transaction. + */ + public boolean isCheckpoint(long version) { + return Arrays.binarySearch(checkpointWritePointers, version) >= 0; + } + + /** + * Returns whether or not the given version should be visible to the current transaction. A version will be visible + * if it was successfully committed prior to the current transaction starting, or was written by the current + * transaction (using either the current write pointer or the write pointer from a prior checkpoint). + * + * @param version the data version to check for visibility + * @return true if the version is visible, false if it should be hidden (filtered) + * + * @see #isVisible(long, boolean) to exclude the current write pointer from visible versions. This method always + * includes the current write pointer. + */ public boolean isVisible(long version) { + return isVisible(version, true); + } + + /** + * Returns whether or not the given version should be visible to the current transaction. A version will be visible + * if it was successfully committed prior to the current transaction starting, or was written by the current + * transaction (using either the current write pointer or the write pointer from a prior checkpoint). + * + * @param version the data version to check for visibility + * @param excludeCurrentWritePointer whether writes from the current write pointer should be visible + * @return true if the version is visible, false if it should be hidden (filtered) + */ + public boolean isVisible(long version, boolean excludeCurrentWritePointer) { // either it was committed before or the change belongs to current tx - return (version <= getReadPointer() && !isExcluded(version)) || writePointer == version; + return (version <= getReadPointer() && !isExcluded(version)) || + ((writePointer == version || isCheckpoint(version)) && + (!excludeCurrentWritePointer || currentWritePointer != version)); } public boolean hasExcludes() { @@ -131,6 +222,15 @@ public int excludesSize() { return invalids.length + inProgress.length; } + /** + * Returns any prior write pointers used in the current transaction. A new write pointer is issued when the + * {@code TransactionContext.checkpoint(Transaction)} operation is called, and the prior write pointer is added + * to the array of checkpoint write pointers. +` */ + public long[] getCheckpointWritePointers() { + return checkpointWritePointers; + } + @Override public String toString() { return new StringBuilder(100) @@ -138,10 +238,12 @@ public String toString() { .append('{') .append("readPointer: ").append(readPointer) .append(", writePointer: ").append(writePointer) + .append(", currentWritePointer: ").append(currentWritePointer) .append(", invalids: ").append(Arrays.toString(invalids)) .append(", inProgress: ").append(Arrays.toString(inProgress)) .append(", firstShortInProgress: ").append(firstShortInProgress) .append(", type: ").append(type) + .append(", checkpointWritePointers: ").append(Arrays.toString(checkpointWritePointers)) .append('}') .toString(); } diff --git a/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java b/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java index a228faac..73d1d0b1 100644 --- a/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java +++ b/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java @@ -18,6 +18,8 @@ import com.google.common.base.Charsets; import com.google.common.base.Objects; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.primitives.Bytes; import com.google.common.primitives.UnsignedBytes; @@ -25,7 +27,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -35,7 +37,8 @@ */ public abstract class AbstractTransactionAwareTable implements TransactionAware { protected final TransactionCodec txCodec; - protected final Set changeSet; + // map of write pointers to change set assocaited with each + protected final Map> changeSets; protected final TxConstants.ConflictDetection conflictLevel; protected Transaction tx; protected boolean allowNonTransactional; @@ -44,7 +47,7 @@ public AbstractTransactionAwareTable(TxConstants.ConflictDetection conflictLevel this.conflictLevel = conflictLevel; this.allowNonTransactional = allowNonTransactional; this.txCodec = new TransactionCodec(); - this.changeSet = new HashSet(); + this.changeSets = Maps.newHashMap(); } /** @@ -75,8 +78,10 @@ public Collection getTxChanges() { } Collection txChanges = new TreeSet(UnsignedBytes.lexicographicalComparator()); - for (ActionChange change : changeSet) { - txChanges.add(getChangeKey(change.getRow(), change.getFamily(), change.getQualifier())); + for (Set changeSet : changeSets.values()) { + for (ActionChange change : changeSet) { + txChanges.add(getChangeKey(change.getRow(), change.getFamily(), change.getQualifier())); + } } return txChanges; } @@ -111,7 +116,7 @@ public boolean commitTx() throws Exception { @Override public void postTxCommit() { tx = null; - changeSet.clear(); + changeSets.clear(); } @Override @@ -137,6 +142,12 @@ public boolean rollbackTx() throws Exception { protected abstract boolean doRollback() throws Exception; protected void addToChangeSet(byte[] row, byte[] family, byte[] qualifier) { + long currentWritePointer = tx.getCurrentWritePointer(); + Set changeSet = changeSets.get(currentWritePointer); + if (changeSet == null) { + changeSet = Sets.newHashSet(); + changeSets.put(currentWritePointer, changeSet); + } switch (conflictLevel) { case ROW: case NONE: diff --git a/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java b/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java index d9a760ef..86e74d05 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java +++ b/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java @@ -58,6 +58,15 @@ public void addTransactionAware(TransactionAware txAware) { } } + /** + * Starts a new transaction. Calling this will initiate a new transaction using the {@link TransactionSystemClient}, + * and pass the returned transaction to {@link TransactionAware#startTx(Transaction)} for each registered + * TransactionAware. If an exception is encountered, the transaction will be aborted and a + * {@code TransactionFailureException} wrapping the root cause will be thrown. + * + * @throws TransactionFailureException if an exception occurs starting the transaction with any registered + * TransactionAware + */ public void start() throws TransactionFailureException { currentTx = txClient.startShort(); for (TransactionAware txAware : txAwares) { @@ -73,6 +82,14 @@ public void start() throws TransactionFailureException { } } + /** + * Commits the current transaction. This will: check for any conflicts, based on the change set aggregated from + * all registered {@link TransactionAware} instances; flush any pending writes from the {@code TransactionAware}s; + * commit the current transaction with the {@link TransactionSystemClient}; and clear the current transaction state. + * + * @throws TransactionConflictException if a conflict is detected with a recently committed transaction + * @throws TransactionFailureException if an error occurs while committing + */ public void finish() throws TransactionFailureException { Preconditions.checkState(currentTx != null, "Cannot finish tx that has not been started"); // each of these steps will abort and rollback the tx in case if errors, and throw an exception @@ -83,10 +100,52 @@ public void finish() throws TransactionFailureException { currentTx = null; } + /** + * Aborts the given transaction, and rolls back all data set changes. If rollback fails, + * the transaction is invalidated. If an exception is caught during rollback, the exception + * is rethrown wrapped in a TransactionFailureException, after all remaining TransactionAwares have + * completed rollback. + * + * @throws TransactionFailureException for any exception that is encountered. + */ public void abort() throws TransactionFailureException { abort(null); } + /** + * Checkpoints the current transaction by flushing any pending writes for the registered {@link TransactionAware} + * instances, and obtaining a new current write pointer for the transaction. By performing a checkpoint, + * the client can ensure that all previous writes were flushed and are visible. By default, the current write + * pointer for the transaction is also visible. The current write pointer can be excluded from read + * operations by setting an attribute with the key {@link TxConstants#TX_EXCLUDE_CURRENT_WRITE} on the + * {@code Get} or {@code Scan} operation. After the checkpoint operation is performed, the updated + * {@link Transaction} instance will be passed to {@link TransactionAware#startTx(Transaction)} for each + * registered {@code TransactionAware} instance. + * + * @throws TransactionFailureException if an error occurs while performing the checkpoint + */ + public void checkpoint() throws TransactionFailureException { + Preconditions.checkState(currentTx != null, "Cannot checkpoint tx that has not been started"); + persist(); + try { + currentTx = txClient.checkpoint(currentTx); + // update the current transaction with all TransactionAwares + for (TransactionAware txAware : txAwares) { + txAware.startTx(currentTx); + } + } catch (TransactionNotInProgressException e) { + String message = String.format("Transaction %d is not in progress.", currentTx.getWritePointer()); + LOG.warn(message, e); + abort(new TransactionFailureException(message, e)); + // abort will throw that exception + } catch (Throwable e) { + String message = String.format("Exception from checkpoint for transaction %d.", currentTx.getWritePointer()); + LOG.warn(message, e); + abort(new TransactionFailureException(message, e)); + // abort will throw that exception + } + } + /** * Returns the current transaction or null if no transaction is currently in progress. */ @@ -99,7 +158,7 @@ public Transaction getCurrentTransaction() { /** * Aborts the given transaction, and rolls back all data set changes. If rollback fails, * the transaction is invalidated. If an exception is caught during rollback, the exception - * is rethrown wrapped into a TransactionFailureException, after all remaining datasets have + * is rethrown wrapped into a TransactionFailureException, after all remaining TransactionAwares have * completed rollback. If an existing exception is passed in, that exception is thrown in either * case, whether the rollback is successful or not. In other words, this method always throws the * first exception that it encounters. diff --git a/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java b/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java index ecdf1312..54580fdb 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java +++ b/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java @@ -1,5 +1,5 @@ /* - * Copyright © 2012-2014 Cask Data, Inc. + * Copyright © 2012-2015 Cask Data, Inc. * * Licensed 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 @@ -518,12 +518,13 @@ public static Map txnBackwardsCompatCheck(int defaultLongTim // handle null expiration long newExpiration = getTxExpirationFromWritePointer(writePointer, defaultLongTimeout); InProgressTx compatTx = - new InProgressTx(entry.getValue().getVisibilityUpperBound(), newExpiration, TransactionType.LONG); + new InProgressTx(entry.getValue().getVisibilityUpperBound(), newExpiration, TransactionType.LONG, + entry.getValue().getCheckpointWritePointers()); entry.setValue(compatTx); } else if (entry.getValue().getType() == null) { InProgressTx compatTx = new InProgressTx(entry.getValue().getVisibilityUpperBound(), entry.getValue().getExpiration(), - TransactionType.SHORT); + TransactionType.SHORT, entry.getValue().getCheckpointWritePointers()); entry.setValue(compatTx); } } @@ -613,7 +614,7 @@ private void replayLogs(Collection logs) { break; } } - doAbort(edit.getWritePointer(), type); + doAbort(edit.getWritePointer(), edit.getCheckpointPointers(), type); break; case TRUNCATE_INVALID_TX: if (edit.getTruncateInvalidTxTime() != 0) { @@ -622,6 +623,9 @@ private void replayLogs(Collection logs) { doTruncateInvalidTx(edit.getTruncateInvalidTx()); } break; + case CHECKPOINT: + doCheckpoint(edit.getWritePointer(), edit.getParentWritePointer()); + break; default: // unknown type! throw new IllegalArgumentException("Invalid state for WAL entry: " + edit.getState()); @@ -766,6 +770,10 @@ private Transaction startTx(long expiration, TransactionType type) { private void addInProgressAndAdvance(long writePointer, long visibilityUpperBound, long expiration, TransactionType type) { inProgress.put(writePointer, new InProgressTx(visibilityUpperBound, expiration, type)); + advanceWritePointer(writePointer); + } + + private void advanceWritePointer(long writePointer) { // don't move the write pointer back if we have out of order transaction log entries if (writePointer > lastWritePointer) { lastWritePointer = writePointer; @@ -908,16 +916,16 @@ public void abort(Transaction tx) { try { synchronized (this) { ensureAvailable(); - doAbort(tx.getWritePointer(), tx.getType()); + doAbort(tx.getWritePointer(), tx.getCheckpointWritePointers(), tx.getType()); } - appendToLog(TransactionEdit.createAborted(tx.getWritePointer(), tx.getType())); + appendToLog(TransactionEdit.createAborted(tx.getWritePointer(), tx.getType(), tx.getCheckpointWritePointers())); txMetricsCollector.histogram("abort.latency", (int) timer.elapsedMillis()); } finally { this.logReadLock.unlock(); } } - private void doAbort(long writePointer, TransactionType type) { + private void doAbort(long writePointer, long[] checkpointWritePointers, TransactionType type) { committingChangeSets.remove(writePointer); if (type == TransactionType.LONG) { @@ -933,6 +941,13 @@ private void doAbort(long writePointer, TransactionType type) { if (removed == null) { // tx was not in progress! perhaps it timed out and is invalid? try to remove it there. if (invalid.rem(writePointer)) { + // remove any invalidated checkpoint pointers + // this will only be present if the parent write pointer was also invalidated + if (checkpointWritePointers != null) { + for (int i = 0; i < checkpointWritePointers.length; i++) { + invalid.rem(checkpointWritePointers[i]); + } + } invalidArray = invalid.toLongArray(); LOG.info("Tx invalid list: removed aborted tx {}", writePointer); // removed a tx from excludes: must move read pointer @@ -969,11 +984,19 @@ private boolean doInvalidate(long writePointer) { InProgressTx previous = inProgress.remove(writePointer); // This check is to prevent from invalidating committed transactions if (previous != null || previousChangeSet != null) { + // add tx to invalids + invalid.add(writePointer); if (previous == null) { LOG.debug("Invalidating tx {} in committing change sets but not in-progress", writePointer); + } else { + // invalidate any checkpoint write pointers + LongArrayList childWritePointers = previous.getCheckpointWritePointers(); + if (childWritePointers != null) { + for (int i = 0; i < childWritePointers.size(); i++) { + invalid.add(childWritePointers.get(i)); + } + } } - // add tx to invalids - invalid.add(writePointer); LOG.info("Tx invalid list: added tx {} because of invalidate", writePointer); // todo: find a more efficient way to keep this sorted. Could it just be an array? Collections.sort(invalid); @@ -1064,6 +1087,53 @@ private boolean doTruncateInvalidTxBefore(long time) throws InvalidTruncateTimeE } return doTruncateInvalidTx(toTruncate); } + + public Transaction checkpoint(Transaction originalTx) throws TransactionNotInProgressException { + txMetricsCollector.rate("checkpoint"); + Stopwatch timer = new Stopwatch().start(); + + Transaction checkpointedTx = null; + long txId = originalTx.getWritePointer(); + long newWritePointer = 0; + // guard against changes to the transaction log while processing + this.logReadLock.lock(); + try { + synchronized (this) { + ensureAvailable(); + // check that the parent tx is in progress + InProgressTx parentTx = inProgress.get(txId); + if (parentTx == null) { + if (invalid.contains(txId)) { + throw new TransactionNotInProgressException( + String.format("Transaction %d is not in progress because it was invalidated", txId)); + } else { + throw new TransactionNotInProgressException( + String.format("Transaction %d is not in progress", txId)); + } + } + newWritePointer = getNextWritePointer(); + doCheckpoint(newWritePointer, txId); + // create a new transaction with the same read snapshot, plus the additional checkpoint write pointer + // the same read snapshot is maintained to + checkpointedTx = new Transaction(originalTx, newWritePointer, + parentTx.getCheckpointWritePointers().toLongArray()); + } + // appending to WAL out of global lock for concurrent performance + // we should still be able to arrive at the same state even if log entries are out of order + appendToLog(TransactionEdit.createCheckpoint(newWritePointer, txId)); + } finally { + this.logReadLock.unlock(); + } + txMetricsCollector.histogram("checkpoint.latency", (int) timer.elapsedMillis()); + + return checkpointedTx; + } + + private void doCheckpoint(long newWritePointer, long parentWritePointer) { + InProgressTx existingTx = inProgress.get(parentWritePointer); + existingTx.addCheckpointWritePointer(newWritePointer); + advanceWritePointer(newWritePointer); + } // hack for exposing important metric public int getExcludedListSize() { @@ -1129,17 +1199,23 @@ private void moveReadPointerIfNeeded(long committedWritePointer) { private Transaction createTransaction(long writePointer, TransactionType type) { // For holding the first in progress short transaction Id (with timeout >= 0). long firstShortTx = Transaction.NO_TX_IN_PROGRESS; - long[] array = new long[inProgress.size()]; - int i = 0; + LongArrayList inProgressIds = new LongArrayList(inProgress.size()); for (Map.Entry entry : inProgress.entrySet()) { long txId = entry.getKey(); - array[i++] = txId; + inProgressIds.add(txId); + // add any checkpointed write pointers to the in-progress list + LongArrayList childIds = entry.getValue().getCheckpointWritePointers(); + if (childIds != null) { + for (int i = 0; i < childIds.size(); i++) { + inProgressIds.add(childIds.get(i)); + } + } if (firstShortTx == Transaction.NO_TX_IN_PROGRESS && !entry.getValue().isLongRunning()) { firstShortTx = txId; } } - return new Transaction(readPointer, writePointer, invalidArray, array, firstShortTx, type); + return new Transaction(readPointer, writePointer, invalidArray, inProgressIds.toLongArray(), firstShortTx, type); } private void appendToLog(TransactionEdit edit) { @@ -1224,11 +1300,18 @@ public static final class InProgressTx { private final long visibilityUpperBound; private final long expiration; private final TransactionType type; + private LongArrayList checkpointWritePointers = new LongArrayList(); public InProgressTx(long visibilityUpperBound, long expiration, TransactionType type) { + this(visibilityUpperBound, expiration, type, new LongArrayList()); + } + + public InProgressTx(long visibilityUpperBound, long expiration, TransactionType type, + LongArrayList checkpointWritePointers) { this.visibilityUpperBound = visibilityUpperBound; this.expiration = expiration; this.type = type; + this.checkpointWritePointers = checkpointWritePointers; } // For backwards compatibility when long running txns were represented with -1 expiration @@ -1258,6 +1341,14 @@ public boolean isLongRunning() { return type == TransactionType.LONG; } + public void addCheckpointWritePointer(long checkpointWritePointer) { + checkpointWritePointers.add(checkpointWritePointer); + } + + public LongArrayList getCheckpointWritePointers() { + return checkpointWritePointers; + } + @Override public boolean equals(Object o) { if (o == null || !(o instanceof InProgressTx)) { @@ -1270,12 +1361,14 @@ public boolean equals(Object o) { InProgressTx other = (InProgressTx) o; return Objects.equal(visibilityUpperBound, other.getVisibilityUpperBound()) && - Objects.equal(expiration, other.getExpiration()) && Objects.equal(type, other.type); + Objects.equal(expiration, other.getExpiration()) && + Objects.equal(type, other.type) && + Objects.equal(checkpointWritePointers, other.checkpointWritePointers); } @Override public int hashCode() { - return Objects.hashCode(visibilityUpperBound, expiration, type); + return Objects.hashCode(visibilityUpperBound, expiration, type, checkpointWritePointers); } @Override @@ -1287,4 +1380,5 @@ public String toString() { .toString(); } } + } diff --git a/tephra-core/src/main/java/co/cask/tephra/TransactionSystemClient.java b/tephra-core/src/main/java/co/cask/tephra/TransactionSystemClient.java index 069b5968..2d7d3db4 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TransactionSystemClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/TransactionSystemClient.java @@ -93,6 +93,14 @@ public interface TransactionSystemClient { */ boolean invalidate(long tx); + /** + * Performs a checkpoint operation on the current transaction, returning a new Transaction instance with the + * updated state. A checkpoint operation assigns a new write pointer for the current transaction. + * @param tx the current transaction to checkpoint + * @return an updated transaction instance with the new write pointer + */ + Transaction checkpoint(Transaction tx) throws TransactionNotInProgressException; + /** * Retrieves the state of the transaction manager and send it as a stream. The snapshot will not be persisted. * @return an input stream containing an encoded snapshot of the transaction manager diff --git a/tephra-core/src/main/java/co/cask/tephra/TxConstants.java b/tephra-core/src/main/java/co/cask/tephra/TxConstants.java index 2bcafc48..e26f585b 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TxConstants.java +++ b/tephra-core/src/main/java/co/cask/tephra/TxConstants.java @@ -18,6 +18,8 @@ import co.cask.tephra.snapshot.DefaultSnapshotCodec; import co.cask.tephra.snapshot.SnapshotCodecV2; +import co.cask.tephra.snapshot.SnapshotCodecV3; +import co.cask.tephra.snapshot.SnapshotCodecV4; import java.util.concurrent.TimeUnit; @@ -95,6 +97,11 @@ public enum ConflictDetection { */ public static final byte[] FAMILY_DELETE_QUALIFIER = new byte[0]; + /** + * Attribute key used to exclude the transaction's current write pointer from reads. + */ + public static final String TX_EXCLUDE_CURRENT_WRITE = "cask.tx.exclude.current"; + // Constants for monitoring status public static final String STATUS_OK = "OK"; public static final String STATUS_NOTOK = "NOTOK"; @@ -305,7 +312,7 @@ public static final class Persist { */ public static final String CFG_TX_SNAPHOT_CODEC_CLASSES = "data.tx.snapshot.codecs"; public static final Class[] DEFAULT_TX_SNAPHOT_CODEC_CLASSES = - { DefaultSnapshotCodec.class, SnapshotCodecV2.class }; + { DefaultSnapshotCodec.class, SnapshotCodecV2.class, SnapshotCodecV3.class, SnapshotCodecV4.class }; } } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java index c5006ef7..dd83da4d 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java @@ -30,13 +30,15 @@ public final class TransactionConverterUtils { public static TTransaction wrap(Transaction tx) { return new TTransaction(tx.getWritePointer(), tx.getReadPointer(), Longs.asList(tx.getInvalids()), Longs.asList(tx.getInProgress()), - tx.getFirstShortInProgress(), getTTransactionType(tx.getType())); + tx.getFirstShortInProgress(), getTTransactionType(tx.getType()), + tx.getCurrentWritePointer(), Longs.asList(tx.getCheckpointWritePointers())); } public static Transaction unwrap(TTransaction thriftTx) { - return new Transaction(thriftTx.getReadPointer(), thriftTx.getWritePointer(), + return new Transaction(thriftTx.getReadPointer(), thriftTx.getWritePointer(), thriftTx.getCurrentWritePointer(), Longs.toArray(thriftTx.getInvalids()), Longs.toArray(thriftTx.getInProgress()), - thriftTx.getFirstShort(), getTransactionType(thriftTx.getType())); + thriftTx.getFirstShort(), getTransactionType(thriftTx.getType()), + Longs.toArray(thriftTx.getCheckpointWritePointers())); } private static TransactionType getTransactionType(TTransactionType tType) { diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java index b87a2f26..a3387d2d 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java @@ -379,6 +379,24 @@ public Boolean execute(TransactionServiceThriftClient client) } } + @Override + public Transaction checkpoint(final Transaction tx) throws TransactionNotInProgressException { + try { + return this.execute( + new Operation("checkpoint") { + @Override + Transaction execute(TransactionServiceThriftClient client) throws Exception { + return client.checkpoint(tx); + } + } + ); + } catch (TransactionNotInProgressException te) { + throw te; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + @Override public InputStream getSnapshotInputStream() throws TransactionCouldNotTakeSnapshotException { try { diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftClient.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftClient.java index 9bfe4c0f..ff2c8b03 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftClient.java @@ -109,6 +109,10 @@ public boolean invalidate(long tx) throws TException { return client.invalidateTx(tx); } + public Transaction checkpoint(Transaction tx) throws TException { + return TransactionConverterUtils.unwrap(client.checkpoint(TransactionConverterUtils.wrap(tx))); + } + public InputStream getSnapshotStream() throws TException { ByteBuffer buffer = client.getSnapshot(); if (buffer.hasArray()) { diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftHandler.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftHandler.java index d97f774a..3329eabd 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftHandler.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceThriftHandler.java @@ -1,5 +1,5 @@ /* - * Copyright © 2012-2014 Cask Data, Inc. + * Copyright © 2012-2015 Cask Data, Inc. * * Licensed 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 @@ -110,16 +110,6 @@ public boolean invalidateTx(long tx) throws TException { return txManager.invalidate(tx); } - @Override - public void init() throws Exception { - txManager.startAndWait(); - } - - @Override - public void destroy() throws Exception { - txManager.stopAndWait(); - } - @Override public ByteBuffer getSnapshot() throws TException { try { @@ -167,4 +157,26 @@ public TBoolean truncateInvalidTxBefore(long time) throws TException { public int invalidTxSize() throws TException { return txManager.getInvalidSize(); } + + @Override + public TTransaction checkpoint(TTransaction originalTx) throws TException { + try { + return TransactionConverterUtils.wrap( + txManager.checkpoint(TransactionConverterUtils.unwrap(originalTx))); + } catch (TransactionNotInProgressException e) { + throw new TTransactionNotInProgressException(e.getMessage()); + } + } + + /* RPCServiceHandler implementation */ + + @Override + public void init() throws Exception { + txManager.startAndWait(); + } + + @Override + public void destroy() throws Exception { + txManager.stopAndWait(); + } } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TBoolean.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TBoolean.java index 7ed95802..c31b0da4 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TBoolean.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TBoolean.java @@ -22,19 +22,29 @@ */ package co.cask.tephra.distributed.thrift; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; -import org.apache.thrift.scheme.TupleScheme; -import java.util.BitSet; -import java.util.Collections; +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TBoolean implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolean"); diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TInvalidTruncateTimeException.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TInvalidTruncateTimeException.java index 2f6a761d..1daa166d 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TInvalidTruncateTimeException.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TInvalidTruncateTimeException.java @@ -22,19 +22,29 @@ */ package co.cask.tephra.distributed.thrift; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; -import org.apache.thrift.scheme.TupleScheme; -import java.util.BitSet; -import java.util.Collections; +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TInvalidTruncateTimeException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TInvalidTruncateTimeException"); diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java index a9d51d85..2d7cc79f 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java @@ -22,21 +22,29 @@ */ package co.cask.tephra.distributed.thrift; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; -import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; import java.util.ArrayList; -import java.util.BitSet; -import java.util.Collections; +import java.util.Map; +import java.util.HashMap; import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; import java.util.EnumSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TTransaction implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTransaction"); @@ -47,6 +55,8 @@ public class TTransaction implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -64,6 +74,8 @@ public class TTransaction implements org.apache.thrift.TBase checkpointWritePointers; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -76,7 +88,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { * * @see TTransactionType */ - TYPE((short)6, "type"); + TYPE((short)6, "type"), + CURRENT_WRITE_POINTER((short)7, "currentWritePointer"), + CHECKPOINT_WRITE_POINTERS((short)8, "checkpointWritePointers"); private static final Map byName = new HashMap(); @@ -103,6 +117,10 @@ public static _Fields findByThriftId(int fieldId) { return FIRST_SHORT; case 6: // TYPE return TYPE; + case 7: // CURRENT_WRITE_POINTER + return CURRENT_WRITE_POINTER; + case 8: // CHECKPOINT_WRITE_POINTERS + return CHECKPOINT_WRITE_POINTERS; default: return null; } @@ -146,6 +164,7 @@ public String getFieldName() { private static final int __WRITEPOINTER_ISSET_ID = 0; private static final int __READPOINTER_ISSET_ID = 1; private static final int __FIRSTSHORT_ISSET_ID = 2; + private static final int __CURRENTWRITEPOINTER_ISSET_ID = 3; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -164,6 +183,11 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTransactionType.class))); + tmpMap.put(_Fields.CURRENT_WRITE_POINTER, new org.apache.thrift.meta_data.FieldMetaData("currentWritePointer", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.CHECKPOINT_WRITE_POINTERS, new org.apache.thrift.meta_data.FieldMetaData("checkpointWritePointers", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTransaction.class, metaDataMap); } @@ -177,7 +201,9 @@ public TTransaction( List invalids, List inProgress, long firstShort, - TTransactionType type) + TTransactionType type, + long currentWritePointer, + List checkpointWritePointers) { this(); this.writePointer = writePointer; @@ -189,6 +215,9 @@ public TTransaction( this.firstShort = firstShort; setFirstShortIsSet(true); this.type = type; + this.currentWritePointer = currentWritePointer; + setCurrentWritePointerIsSet(true); + this.checkpointWritePointers = checkpointWritePointers; } /** @@ -216,6 +245,14 @@ public TTransaction(TTransaction other) { if (other.isSetType()) { this.type = other.type; } + this.currentWritePointer = other.currentWritePointer; + if (other.isSetCheckpointWritePointers()) { + List __this__checkpointWritePointers = new ArrayList(); + for (Long other_element : other.checkpointWritePointers) { + __this__checkpointWritePointers.add(other_element); + } + this.checkpointWritePointers = __this__checkpointWritePointers; + } } public TTransaction deepCopy() { @@ -233,6 +270,9 @@ public void clear() { setFirstShortIsSet(false); this.firstShort = 0; this.type = null; + setCurrentWritePointerIsSet(false); + this.currentWritePointer = 0; + this.checkpointWritePointers = null; } public long getWritePointer() { @@ -414,6 +454,68 @@ public void setTypeIsSet(boolean value) { } } + public long getCurrentWritePointer() { + return this.currentWritePointer; + } + + public TTransaction setCurrentWritePointer(long currentWritePointer) { + this.currentWritePointer = currentWritePointer; + setCurrentWritePointerIsSet(true); + return this; + } + + public void unsetCurrentWritePointer() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CURRENTWRITEPOINTER_ISSET_ID); + } + + /** Returns true if field currentWritePointer is set (has been assigned a value) and false otherwise */ + public boolean isSetCurrentWritePointer() { + return EncodingUtils.testBit(__isset_bitfield, __CURRENTWRITEPOINTER_ISSET_ID); + } + + public void setCurrentWritePointerIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CURRENTWRITEPOINTER_ISSET_ID, value); + } + + public int getCheckpointWritePointersSize() { + return (this.checkpointWritePointers == null) ? 0 : this.checkpointWritePointers.size(); + } + + public java.util.Iterator getCheckpointWritePointersIterator() { + return (this.checkpointWritePointers == null) ? null : this.checkpointWritePointers.iterator(); + } + + public void addToCheckpointWritePointers(long elem) { + if (this.checkpointWritePointers == null) { + this.checkpointWritePointers = new ArrayList(); + } + this.checkpointWritePointers.add(elem); + } + + public List getCheckpointWritePointers() { + return this.checkpointWritePointers; + } + + public TTransaction setCheckpointWritePointers(List checkpointWritePointers) { + this.checkpointWritePointers = checkpointWritePointers; + return this; + } + + public void unsetCheckpointWritePointers() { + this.checkpointWritePointers = null; + } + + /** Returns true if field checkpointWritePointers is set (has been assigned a value) and false otherwise */ + public boolean isSetCheckpointWritePointers() { + return this.checkpointWritePointers != null; + } + + public void setCheckpointWritePointersIsSet(boolean value) { + if (!value) { + this.checkpointWritePointers = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case WRITE_POINTER: @@ -428,7 +530,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetReadPointer(); } else { - setReadPointer((Long) value); + setReadPointer((Long)value); } break; @@ -436,7 +538,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetInvalids(); } else { - setInvalids((List) value); + setInvalids((List)value); } break; @@ -444,7 +546,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetInProgress(); } else { - setInProgress((List) value); + setInProgress((List)value); } break; @@ -452,7 +554,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetFirstShort(); } else { - setFirstShort((Long) value); + setFirstShort((Long)value); } break; @@ -460,7 +562,23 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetType(); } else { - setType((TTransactionType) value); + setType((TTransactionType)value); + } + break; + + case CURRENT_WRITE_POINTER: + if (value == null) { + unsetCurrentWritePointer(); + } else { + setCurrentWritePointer((Long)value); + } + break; + + case CHECKPOINT_WRITE_POINTERS: + if (value == null) { + unsetCheckpointWritePointers(); + } else { + setCheckpointWritePointers((List)value); } break; @@ -487,6 +605,12 @@ public Object getFieldValue(_Fields field) { case TYPE: return getType(); + case CURRENT_WRITE_POINTER: + return Long.valueOf(getCurrentWritePointer()); + + case CHECKPOINT_WRITE_POINTERS: + return getCheckpointWritePointers(); + } throw new IllegalStateException(); } @@ -510,6 +634,10 @@ public boolean isSet(_Fields field) { return isSetFirstShort(); case TYPE: return isSetType(); + case CURRENT_WRITE_POINTER: + return isSetCurrentWritePointer(); + case CHECKPOINT_WRITE_POINTERS: + return isSetCheckpointWritePointers(); } throw new IllegalStateException(); } @@ -581,6 +709,24 @@ public boolean equals(TTransaction that) { return false; } + boolean this_present_currentWritePointer = true; + boolean that_present_currentWritePointer = true; + if (this_present_currentWritePointer || that_present_currentWritePointer) { + if (!(this_present_currentWritePointer && that_present_currentWritePointer)) + return false; + if (this.currentWritePointer != that.currentWritePointer) + return false; + } + + boolean this_present_checkpointWritePointers = true && this.isSetCheckpointWritePointers(); + boolean that_present_checkpointWritePointers = true && that.isSetCheckpointWritePointers(); + if (this_present_checkpointWritePointers || that_present_checkpointWritePointers) { + if (!(this_present_checkpointWritePointers && that_present_checkpointWritePointers)) + return false; + if (!this.checkpointWritePointers.equals(that.checkpointWritePointers)) + return false; + } + return true; } @@ -657,6 +803,26 @@ public int compareTo(TTransaction other) { return lastComparison; } } + lastComparison = Boolean.valueOf(isSetCurrentWritePointer()).compareTo(typedOther.isSetCurrentWritePointer()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCurrentWritePointer()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.currentWritePointer, typedOther.currentWritePointer); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCheckpointWritePointers()).compareTo(typedOther.isSetCheckpointWritePointers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCheckpointWritePointers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.checkpointWritePointers, typedOther.checkpointWritePointers); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -712,6 +878,18 @@ public String toString() { sb.append(this.type); } first = false; + if (!first) sb.append(", "); + sb.append("currentWritePointer:"); + sb.append(this.currentWritePointer); + first = false; + if (!first) sb.append(", "); + sb.append("checkpointWritePointers:"); + if (this.checkpointWritePointers == null) { + sb.append("null"); + } else { + sb.append(this.checkpointWritePointers); + } + first = false; sb.append(")"); return sb.toString(); } @@ -825,6 +1003,32 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TTransaction struct org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 7: // CURRENT_WRITE_POINTER + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.currentWritePointer = iprot.readI64(); + struct.setCurrentWritePointerIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // CHECKPOINT_WRITE_POINTERS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list6 = iprot.readListBegin(); + struct.checkpointWritePointers = new ArrayList(_list6.size); + for (int _i7 = 0; _i7 < _list6.size; ++_i7) + { + long _elem8; // required + _elem8 = iprot.readI64(); + struct.checkpointWritePointers.add(_elem8); + } + iprot.readListEnd(); + } + struct.setCheckpointWritePointersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -850,9 +1054,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TTransaction struc oprot.writeFieldBegin(INVALIDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.invalids.size())); - for (long _iter6 : struct.invalids) + for (long _iter9 : struct.invalids) { - oprot.writeI64(_iter6); + oprot.writeI64(_iter9); } oprot.writeListEnd(); } @@ -862,9 +1066,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TTransaction struc oprot.writeFieldBegin(IN_PROGRESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.inProgress.size())); - for (long _iter7 : struct.inProgress) + for (long _iter10 : struct.inProgress) { - oprot.writeI64(_iter7); + oprot.writeI64(_iter10); } oprot.writeListEnd(); } @@ -878,6 +1082,21 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TTransaction struc oprot.writeI32(struct.type.getValue()); oprot.writeFieldEnd(); } + oprot.writeFieldBegin(CURRENT_WRITE_POINTER_FIELD_DESC); + oprot.writeI64(struct.currentWritePointer); + oprot.writeFieldEnd(); + if (struct.checkpointWritePointers != null) { + oprot.writeFieldBegin(CHECKPOINT_WRITE_POINTERS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.checkpointWritePointers.size())); + for (long _iter11 : struct.checkpointWritePointers) + { + oprot.writeI64(_iter11); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -914,7 +1133,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct if (struct.isSetType()) { optionals.set(5); } - oprot.writeBitSet(optionals, 6); + if (struct.isSetCurrentWritePointer()) { + optionals.set(6); + } + if (struct.isSetCheckpointWritePointers()) { + optionals.set(7); + } + oprot.writeBitSet(optionals, 8); if (struct.isSetWritePointer()) { oprot.writeI64(struct.writePointer); } @@ -924,18 +1149,18 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct if (struct.isSetInvalids()) { { oprot.writeI32(struct.invalids.size()); - for (long _iter8 : struct.invalids) + for (long _iter12 : struct.invalids) { - oprot.writeI64(_iter8); + oprot.writeI64(_iter12); } } } if (struct.isSetInProgress()) { { oprot.writeI32(struct.inProgress.size()); - for (long _iter9 : struct.inProgress) + for (long _iter13 : struct.inProgress) { - oprot.writeI64(_iter9); + oprot.writeI64(_iter13); } } } @@ -945,12 +1170,24 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct if (struct.isSetType()) { oprot.writeI32(struct.type.getValue()); } + if (struct.isSetCurrentWritePointer()) { + oprot.writeI64(struct.currentWritePointer); + } + if (struct.isSetCheckpointWritePointers()) { + { + oprot.writeI32(struct.checkpointWritePointers.size()); + for (long _iter14 : struct.checkpointWritePointers) + { + oprot.writeI64(_iter14); + } + } + } } @Override public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(6); + BitSet incoming = iprot.readBitSet(8); if (incoming.get(0)) { struct.writePointer = iprot.readI64(); struct.setWritePointerIsSet(true); @@ -961,26 +1198,26 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list10 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.invalids = new ArrayList(_list10.size); - for (int _i11 = 0; _i11 < _list10.size; ++_i11) + org.apache.thrift.protocol.TList _list15 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.invalids = new ArrayList(_list15.size); + for (int _i16 = 0; _i16 < _list15.size; ++_i16) { - long _elem12; // required - _elem12 = iprot.readI64(); - struct.invalids.add(_elem12); + long _elem17; // required + _elem17 = iprot.readI64(); + struct.invalids.add(_elem17); } } struct.setInvalidsIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list13 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.inProgress = new ArrayList(_list13.size); - for (int _i14 = 0; _i14 < _list13.size; ++_i14) + org.apache.thrift.protocol.TList _list18 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.inProgress = new ArrayList(_list18.size); + for (int _i19 = 0; _i19 < _list18.size; ++_i19) { - long _elem15; // required - _elem15 = iprot.readI64(); - struct.inProgress.add(_elem15); + long _elem20; // required + _elem20 = iprot.readI64(); + struct.inProgress.add(_elem20); } } struct.setInProgressIsSet(true); @@ -993,6 +1230,23 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) struct.type = TTransactionType.findByValue(iprot.readI32()); struct.setTypeIsSet(true); } + if (incoming.get(6)) { + struct.currentWritePointer = iprot.readI64(); + struct.setCurrentWritePointerIsSet(true); + } + if (incoming.get(7)) { + { + org.apache.thrift.protocol.TList _list21 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.checkpointWritePointers = new ArrayList(_list21.size); + for (int _i22 = 0; _i22 < _list21.size; ++_i22) + { + long _elem23; // required + _elem23 = iprot.readI64(); + struct.checkpointWritePointers.add(_elem23); + } + } + struct.setCheckpointWritePointersIsSet(true); + } } } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionCouldNotTakeSnapshotException.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionCouldNotTakeSnapshotException.java index 4f055792..29fe49bb 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionCouldNotTakeSnapshotException.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionCouldNotTakeSnapshotException.java @@ -22,19 +22,29 @@ */ package co.cask.tephra.distributed.thrift; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; -import org.apache.thrift.scheme.TupleScheme; -import java.util.BitSet; -import java.util.Collections; +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TTransactionCouldNotTakeSnapshotException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTransactionCouldNotTakeSnapshotException"); diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionNotInProgressException.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionNotInProgressException.java index 35245087..62e426b1 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionNotInProgressException.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionNotInProgressException.java @@ -22,19 +22,29 @@ */ package co.cask.tephra.distributed.thrift; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; -import org.apache.thrift.scheme.TupleScheme; -import java.util.BitSet; -import java.util.Collections; +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TTransactionNotInProgressException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTransactionNotInProgressException"); diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java index 9e3ace8c..a875cb34 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java @@ -22,24 +22,29 @@ */ package co.cask.tephra.distributed.thrift; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; -import org.apache.thrift.scheme.TupleScheme; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; -import java.util.BitSet; -import java.util.Collections; -import java.util.EnumMap; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TTransactionServer { @@ -71,6 +76,8 @@ public interface Iface { public int invalidTxSize() throws org.apache.thrift.TException; + public TTransaction checkpoint(TTransaction tx) throws TTransactionNotInProgressException, org.apache.thrift.TException; + } public interface AsyncIface { @@ -101,6 +108,8 @@ public interface AsyncIface { public void invalidTxSize(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void checkpoint(TTransaction tx, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -423,6 +432,32 @@ public int recv_invalidTxSize() throws org.apache.thrift.TException throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "invalidTxSize failed: unknown result"); } + public TTransaction checkpoint(TTransaction tx) throws TTransactionNotInProgressException, org.apache.thrift.TException + { + send_checkpoint(tx); + return recv_checkpoint(); + } + + public void send_checkpoint(TTransaction tx) throws org.apache.thrift.TException + { + checkpoint_args args = new checkpoint_args(); + args.setTx(tx); + sendBase("checkpoint", args); + } + + public TTransaction recv_checkpoint() throws TTransactionNotInProgressException, org.apache.thrift.TException + { + checkpoint_result result = new checkpoint_result(); + receiveBase(result, "checkpoint"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkpoint failed: unknown result"); + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -842,6 +877,38 @@ public int getResult() throws org.apache.thrift.TException { } } + public void checkpoint(TTransaction tx, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + checkpoint_call method_call = new checkpoint_call(tx, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class checkpoint_call extends org.apache.thrift.async.TAsyncMethodCall { + private TTransaction tx; + public checkpoint_call(TTransaction tx, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.tx = tx; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("checkpoint", org.apache.thrift.protocol.TMessageType.CALL, 0)); + checkpoint_args args = new checkpoint_args(); + args.setTx(tx); + args.write(prot); + prot.writeMessageEnd(); + } + + public TTransaction getResult() throws TTransactionNotInProgressException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_checkpoint(); + } + } + } public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { @@ -868,6 +935,7 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public checkpoint() { + super("checkpoint"); + } + + public checkpoint_args getEmptyArgsInstance() { + return new checkpoint_args(); + } + + protected boolean isOneway() { + return false; + } + + public checkpoint_result getResult(I iface, checkpoint_args args) throws org.apache.thrift.TException { + checkpoint_result result = new checkpoint_result(); + try { + result.success = iface.checkpoint(args.tx); + } catch (TTransactionNotInProgressException e) { + result.e = e; + } + return result; + } + } + } public static class startLong_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { @@ -3472,13 +3564,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, canCommitTx_args st case 2: // CHANGES if (schemeField.type == org.apache.thrift.protocol.TType.SET) { { - org.apache.thrift.protocol.TSet _set16 = iprot.readSetBegin(); - struct.changes = new HashSet(2*_set16.size); - for (int _i17 = 0; _i17 < _set16.size; ++_i17) + org.apache.thrift.protocol.TSet _set24 = iprot.readSetBegin(); + struct.changes = new HashSet(2*_set24.size); + for (int _i25 = 0; _i25 < _set24.size; ++_i25) { - ByteBuffer _elem18; // required - _elem18 = iprot.readBinary(); - struct.changes.add(_elem18); + ByteBuffer _elem26; // required + _elem26 = iprot.readBinary(); + struct.changes.add(_elem26); } iprot.readSetEnd(); } @@ -3511,9 +3603,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, canCommitTx_args s oprot.writeFieldBegin(CHANGES_FIELD_DESC); { oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.changes.size())); - for (ByteBuffer _iter19 : struct.changes) + for (ByteBuffer _iter27 : struct.changes) { - oprot.writeBinary(_iter19); + oprot.writeBinary(_iter27); } oprot.writeSetEnd(); } @@ -3550,9 +3642,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, canCommitTx_args st if (struct.isSetChanges()) { { oprot.writeI32(struct.changes.size()); - for (ByteBuffer _iter20 : struct.changes) + for (ByteBuffer _iter28 : struct.changes) { - oprot.writeBinary(_iter20); + oprot.writeBinary(_iter28); } } } @@ -3569,13 +3661,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, canCommitTx_args str } if (incoming.get(1)) { { - org.apache.thrift.protocol.TSet _set21 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.changes = new HashSet(2*_set21.size); - for (int _i22 = 0; _i22 < _set21.size; ++_i22) + org.apache.thrift.protocol.TSet _set29 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.changes = new HashSet(2*_set29.size); + for (int _i30 = 0; _i30 < _set29.size; ++_i30) { - ByteBuffer _elem23; // required - _elem23 = iprot.readBinary(); - struct.changes.add(_elem23); + ByteBuffer _elem31; // required + _elem31 = iprot.readBinary(); + struct.changes.add(_elem31); } } struct.setChangesIsSet(true); @@ -8291,13 +8383,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, truncateInvalidTx_a case 1: // TXNS if (schemeField.type == org.apache.thrift.protocol.TType.SET) { { - org.apache.thrift.protocol.TSet _set24 = iprot.readSetBegin(); - struct.txns = new HashSet(2*_set24.size); - for (int _i25 = 0; _i25 < _set24.size; ++_i25) + org.apache.thrift.protocol.TSet _set32 = iprot.readSetBegin(); + struct.txns = new HashSet(2*_set32.size); + for (int _i33 = 0; _i33 < _set32.size; ++_i33) { - long _elem26; // required - _elem26 = iprot.readI64(); - struct.txns.add(_elem26); + long _elem34; // required + _elem34 = iprot.readI64(); + struct.txns.add(_elem34); } iprot.readSetEnd(); } @@ -8325,9 +8417,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, truncateInvalidTx_ oprot.writeFieldBegin(TXNS_FIELD_DESC); { oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.txns.size())); - for (long _iter27 : struct.txns) + for (long _iter35 : struct.txns) { - oprot.writeI64(_iter27); + oprot.writeI64(_iter35); } oprot.writeSetEnd(); } @@ -8358,9 +8450,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, truncateInvalidTx_a if (struct.isSetTxns()) { { oprot.writeI32(struct.txns.size()); - for (long _iter28 : struct.txns) + for (long _iter36 : struct.txns) { - oprot.writeI64(_iter28); + oprot.writeI64(_iter36); } } } @@ -8372,13 +8464,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, truncateInvalidTx_ar BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TSet _set29 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.txns = new HashSet(2*_set29.size); - for (int _i30 = 0; _i30 < _set29.size; ++_i30) + org.apache.thrift.protocol.TSet _set37 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.txns = new HashSet(2*_set37.size); + for (int _i38 = 0; _i38 < _set37.size; ++_i38) { - long _elem31; // required - _elem31 = iprot.readI64(); - struct.txns.add(_elem31); + long _elem39; // required + _elem39 = iprot.readI64(); + struct.txns.add(_elem39); } } struct.setTxnsIsSet(true); @@ -10160,4 +10252,824 @@ public void read(org.apache.thrift.protocol.TProtocol prot, invalidTxSize_result } + public static class checkpoint_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkpoint_args"); + + private static final org.apache.thrift.protocol.TField TX_FIELD_DESC = new org.apache.thrift.protocol.TField("tx", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new checkpoint_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new checkpoint_argsTupleSchemeFactory()); + } + + public TTransaction tx; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TX((short)1, "tx"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TX + return TX; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TX, new org.apache.thrift.meta_data.FieldMetaData("tx", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTransaction.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkpoint_args.class, metaDataMap); + } + + public checkpoint_args() { + } + + public checkpoint_args( + TTransaction tx) + { + this(); + this.tx = tx; + } + + /** + * Performs a deep copy on other. + */ + public checkpoint_args(checkpoint_args other) { + if (other.isSetTx()) { + this.tx = new TTransaction(other.tx); + } + } + + public checkpoint_args deepCopy() { + return new checkpoint_args(this); + } + + @Override + public void clear() { + this.tx = null; + } + + public TTransaction getTx() { + return this.tx; + } + + public checkpoint_args setTx(TTransaction tx) { + this.tx = tx; + return this; + } + + public void unsetTx() { + this.tx = null; + } + + /** Returns true if field tx is set (has been assigned a value) and false otherwise */ + public boolean isSetTx() { + return this.tx != null; + } + + public void setTxIsSet(boolean value) { + if (!value) { + this.tx = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TX: + if (value == null) { + unsetTx(); + } else { + setTx((TTransaction)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TX: + return getTx(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TX: + return isSetTx(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof checkpoint_args) + return this.equals((checkpoint_args)that); + return false; + } + + public boolean equals(checkpoint_args that) { + if (that == null) + return false; + + boolean this_present_tx = true && this.isSetTx(); + boolean that_present_tx = true && that.isSetTx(); + if (this_present_tx || that_present_tx) { + if (!(this_present_tx && that_present_tx)) + return false; + if (!this.tx.equals(that.tx)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + public int compareTo(checkpoint_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + checkpoint_args typedOther = (checkpoint_args)other; + + lastComparison = Boolean.valueOf(isSetTx()).compareTo(typedOther.isSetTx()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTx()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tx, typedOther.tx); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("checkpoint_args("); + boolean first = true; + + sb.append("tx:"); + if (this.tx == null) { + sb.append("null"); + } else { + sb.append(this.tx); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (tx != null) { + tx.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class checkpoint_argsStandardSchemeFactory implements SchemeFactory { + public checkpoint_argsStandardScheme getScheme() { + return new checkpoint_argsStandardScheme(); + } + } + + private static class checkpoint_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, checkpoint_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TX + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tx = new TTransaction(); + struct.tx.read(iprot); + struct.setTxIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, checkpoint_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.tx != null) { + oprot.writeFieldBegin(TX_FIELD_DESC); + struct.tx.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class checkpoint_argsTupleSchemeFactory implements SchemeFactory { + public checkpoint_argsTupleScheme getScheme() { + return new checkpoint_argsTupleScheme(); + } + } + + private static class checkpoint_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, checkpoint_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetTx()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetTx()) { + struct.tx.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, checkpoint_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.tx = new TTransaction(); + struct.tx.read(iprot); + struct.setTxIsSet(true); + } + } + } + + } + + public static class checkpoint_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkpoint_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new checkpoint_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new checkpoint_resultTupleSchemeFactory()); + } + + public TTransaction success; // required + public TTransactionNotInProgressException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTransaction.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkpoint_result.class, metaDataMap); + } + + public checkpoint_result() { + } + + public checkpoint_result( + TTransaction success, + TTransactionNotInProgressException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public checkpoint_result(checkpoint_result other) { + if (other.isSetSuccess()) { + this.success = new TTransaction(other.success); + } + if (other.isSetE()) { + this.e = new TTransactionNotInProgressException(other.e); + } + } + + public checkpoint_result deepCopy() { + return new checkpoint_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public TTransaction getSuccess() { + return this.success; + } + + public checkpoint_result setSuccess(TTransaction success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public TTransactionNotInProgressException getE() { + return this.e; + } + + public checkpoint_result setE(TTransactionNotInProgressException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TTransaction)value); + } + break; + + case E: + if (value == null) { + unsetE(); + } else { + setE((TTransactionNotInProgressException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case E: + return getE(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case E: + return isSetE(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof checkpoint_result) + return this.equals((checkpoint_result)that); + return false; + } + + public boolean equals(checkpoint_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.isSetE(); + boolean that_present_e = true && that.isSetE(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + public int compareTo(checkpoint_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + checkpoint_result typedOther = (checkpoint_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("checkpoint_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class checkpoint_resultStandardSchemeFactory implements SchemeFactory { + public checkpoint_resultStandardScheme getScheme() { + return new checkpoint_resultStandardScheme(); + } + } + + private static class checkpoint_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, checkpoint_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TTransaction(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new TTransactionNotInProgressException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, checkpoint_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class checkpoint_resultTupleSchemeFactory implements SchemeFactory { + public checkpoint_resultTupleScheme getScheme() { + return new checkpoint_resultTupleScheme(); + } + } + + private static class checkpoint_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, checkpoint_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetE()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetE()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, checkpoint_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = new TTransaction(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.e = new TTransactionNotInProgressException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + } + } + + } + } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionType.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionType.java index 503d8f6e..62d9c791 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionType.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionType.java @@ -23,6 +23,10 @@ package co.cask.tephra.distributed.thrift; +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + public enum TTransactionType implements org.apache.thrift.TEnum { SHORT(1), LONG(2); diff --git a/tephra-core/src/main/java/co/cask/tephra/inmemory/DetachedTxSystemClient.java b/tephra-core/src/main/java/co/cask/tephra/inmemory/DetachedTxSystemClient.java index d8ad9a80..f7877a8e 100644 --- a/tephra-core/src/main/java/co/cask/tephra/inmemory/DetachedTxSystemClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/inmemory/DetachedTxSystemClient.java @@ -22,6 +22,7 @@ import co.cask.tephra.TransactionSystemClient; import co.cask.tephra.TransactionType; import co.cask.tephra.TxConstants; +import it.unimi.dsi.fastutil.longs.LongArrayList; import java.io.InputStream; import java.util.Collection; @@ -48,6 +49,14 @@ public class DetachedTxSystemClient implements TransactionSystemClient { @Override public Transaction startShort() { + long wp = getWritePointer(); + // NOTE: -1 here is because we have logic that uses (readpointer + 1) as a "exclusive stop key" in some datasets + return new Transaction( + Long.MAX_VALUE - 1, wp, new long[0], new long[0], + Transaction.NO_TX_IN_PROGRESS, TransactionType.SHORT); + } + + private long getWritePointer() { long wp = generator.incrementAndGet(); // NOTE: using InMemoryTransactionManager.MAX_TX_PER_MS to be at least close to real one long now = System.currentTimeMillis(); @@ -58,10 +67,7 @@ public Transaction startShort() { wp = advanced; } } - // NOTE: -1 here is because we have logic that uses (readpointer + 1) as a "exclusive stop key" in some datasets - return new Transaction( - Long.MAX_VALUE - 1, wp, new long[0], new long[0], - Transaction.NO_TX_IN_PROGRESS, TransactionType.SHORT); + return wp; } @Override @@ -94,6 +100,14 @@ public boolean invalidate(long tx) { return true; } + @Override + public Transaction checkpoint(Transaction tx) { + long newWritePointer = getWritePointer(); + LongArrayList newCheckpointPointers = new LongArrayList(tx.getCheckpointWritePointers()); + newCheckpointPointers.add(newWritePointer); + return new Transaction(tx, newWritePointer, newCheckpointPointers.toLongArray()); + } + @Override public InputStream getSnapshotInputStream() throws TransactionCouldNotTakeSnapshotException { throw new TransactionCouldNotTakeSnapshotException( diff --git a/tephra-core/src/main/java/co/cask/tephra/inmemory/InMemoryTxSystemClient.java b/tephra-core/src/main/java/co/cask/tephra/inmemory/InMemoryTxSystemClient.java index 2bdbb507..4bb0ea22 100644 --- a/tephra-core/src/main/java/co/cask/tephra/inmemory/InMemoryTxSystemClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/inmemory/InMemoryTxSystemClient.java @@ -83,6 +83,11 @@ public boolean invalidate(long tx) { return txManager.invalidate(tx); } + @Override + public Transaction checkpoint(Transaction tx) throws TransactionNotInProgressException { + return txManager.checkpoint(tx); + } + @Override public InputStream getSnapshotInputStream() throws TransactionCouldNotTakeSnapshotException { try { diff --git a/tephra-core/src/main/java/co/cask/tephra/inmemory/MinimalTxSystemClient.java b/tephra-core/src/main/java/co/cask/tephra/inmemory/MinimalTxSystemClient.java index 5478ef9c..40ac4709 100644 --- a/tephra-core/src/main/java/co/cask/tephra/inmemory/MinimalTxSystemClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/inmemory/MinimalTxSystemClient.java @@ -19,6 +19,7 @@ import co.cask.tephra.InvalidTruncateTimeException; import co.cask.tephra.Transaction; import co.cask.tephra.TransactionCouldNotTakeSnapshotException; +import co.cask.tephra.TransactionNotInProgressException; import co.cask.tephra.TransactionSystemClient; import co.cask.tephra.TransactionType; import co.cask.tephra.TxConstants; @@ -72,6 +73,11 @@ public boolean invalidate(long tx) { return true; } + @Override + public Transaction checkpoint(Transaction tx) throws TransactionNotInProgressException { + return tx; + } + @Override public InputStream getSnapshotInputStream() throws TransactionCouldNotTakeSnapshotException { throw new TransactionCouldNotTakeSnapshotException("Not snapshot to take."); diff --git a/tephra-core/src/main/java/co/cask/tephra/persist/TransactionEdit.java b/tephra-core/src/main/java/co/cask/tephra/persist/TransactionEdit.java index 6a4f068d..0d625278 100644 --- a/tephra-core/src/main/java/co/cask/tephra/persist/TransactionEdit.java +++ b/tephra-core/src/main/java/co/cask/tephra/persist/TransactionEdit.java @@ -34,9 +34,12 @@ */ public class TransactionEdit implements Writable { // provides serde for current version + // V4 adds CHECKPOINT edit type + private static final TransactionEditCodec CODEC_V4 = new TransactionEditCodecV3(); + private static final byte V4 = -4; + // provides serde for old but still supported version, should not be used for writing private static final TransactionEditCodec CODEC_V3 = new TransactionEditCodecV3(); private static final byte V3 = -3; - // provides serde for old but still supported version, should not be used for writing private static final TransactionEditCodec CODEC_V2 = new TransactionEditCodecV2(); private static final byte V2 = -2; private static final TransactionEditCodec CODEC_V1 = new TransactionEditCodecV1(); @@ -46,7 +49,7 @@ public class TransactionEdit implements Writable { * The possible state changes for a transaction. */ public enum State { - INPROGRESS, COMMITTING, COMMITTED, INVALID, ABORTED, MOVE_WATERMARK, TRUNCATE_INVALID_TX + INPROGRESS, COMMITTING, COMMITTED, INVALID, ABORTED, MOVE_WATERMARK, TRUNCATE_INVALID_TX, CHECKPOINT } private long writePointer; @@ -66,6 +69,8 @@ public enum State { private TransactionType type; private Set truncateInvalidTx; private long truncateInvalidTxTime; + private long parentWritePointer; + private long[] checkpointPointers; // for Writable public TransactionEdit() { @@ -76,7 +81,8 @@ public TransactionEdit() { // package private for testing TransactionEdit(long writePointer, long visibilityUpperBound, State state, long expirationDate, Set changes, long commitPointer, boolean canCommit, TransactionType type, - Set truncateInvalidTx, long truncateInvalidTxTime) { + Set truncateInvalidTx, long truncateInvalidTxTime, long parentWritePointer, + long[] checkpointPointers) { this.writePointer = writePointer; this.visibilityUpperBound = visibilityUpperBound; this.state = state; @@ -87,6 +93,8 @@ public TransactionEdit() { this.type = type; this.truncateInvalidTx = truncateInvalidTx != null ? truncateInvalidTx : Collections.emptySet(); this.truncateInvalidTxTime = truncateInvalidTxTime; + this.parentWritePointer = parentWritePointer; + this.checkpointPointers = checkpointPointers; } /** @@ -163,20 +171,36 @@ public long getTruncateInvalidTxTime() { return truncateInvalidTxTime; } + /** + * Returns the parent write pointer for a checkpoint operation. This is only populated for edits of type + * {@link State#CHECKPOINT} + */ + public long getParentWritePointer() { + return parentWritePointer; + } + + /** + * Returns the checkpoint write pointers for the edit. This is only populated for edits of type + * {@link State#ABORTED}. + */ + public long[] getCheckpointPointers() { + return checkpointPointers; + } + /** * Creates a new instance in the {@link State#INPROGRESS} state. */ public static TransactionEdit createStarted(long writePointer, long visibilityUpperBound, long expirationDate, TransactionType type) { return new TransactionEdit(writePointer, visibilityUpperBound, State.INPROGRESS, - expirationDate, null, 0L, false, type, null, 0L); + expirationDate, null, 0L, false, type, null, 0L, 0L, null); } /** * Creates a new instance in the {@link State#COMMITTING} state. */ public static TransactionEdit createCommitting(long writePointer, Set changes) { - return new TransactionEdit(writePointer, 0L, State.COMMITTING, 0L, changes, 0L, false, null, null, 0L); + return new TransactionEdit(writePointer, 0L, State.COMMITTING, 0L, changes, 0L, false, null, null, 0L, 0L, null); } /** @@ -185,35 +209,37 @@ public static TransactionEdit createCommitting(long writePointer, Set public static TransactionEdit createCommitted(long writePointer, Set changes, long nextWritePointer, boolean canCommit) { return new TransactionEdit(writePointer, 0L, State.COMMITTED, 0L, changes, nextWritePointer, canCommit, null, - null, 0L); + null, 0L, 0L, null); } /** * Creates a new instance in the {@link State#ABORTED} state. */ - public static TransactionEdit createAborted(long writePointer, TransactionType type) { - return new TransactionEdit(writePointer, 0L, State.ABORTED, 0L, null, 0L, false, type, null, 0L); + public static TransactionEdit createAborted(long writePointer, TransactionType type, long[] checkpointPointers) { + return new TransactionEdit(writePointer, 0L, State.ABORTED, 0L, null, 0L, false, type, null, 0L, 0L, + checkpointPointers); } /** * Creates a new instance in the {@link State#INVALID} state. */ public static TransactionEdit createInvalid(long writePointer) { - return new TransactionEdit(writePointer, 0L, State.INVALID, 0L, null, 0L, false, null, null, 0L); + return new TransactionEdit(writePointer, 0L, State.INVALID, 0L, null, 0L, false, null, null, 0L, 0L, null); } /** * Creates a new instance in the {@link State#MOVE_WATERMARK} state. */ public static TransactionEdit createMoveWatermark(long writePointer) { - return new TransactionEdit(writePointer, 0L, State.MOVE_WATERMARK, 0L, null, 0L, false, null, null, 0L); + return new TransactionEdit(writePointer, 0L, State.MOVE_WATERMARK, 0L, null, 0L, false, null, null, 0L, 0L, null); } /** * Creates a new instance in the {@link State#TRUNCATE_INVALID_TX} state. */ public static TransactionEdit createTruncateInvalidTx(Set truncateInvalidTx) { - return new TransactionEdit(0L, 0L, State.TRUNCATE_INVALID_TX, 0L, null, 0L, false, null, truncateInvalidTx, 0L); + return new TransactionEdit(0L, 0L, State.TRUNCATE_INVALID_TX, 0L, null, 0L, false, null, truncateInvalidTx, + 0L, 0L, null); } /** @@ -221,18 +247,29 @@ public static TransactionEdit createTruncateInvalidTx(Set truncateInvalidT */ public static TransactionEdit createTruncateInvalidTxBefore(long truncateInvalidTxTime) { return new TransactionEdit(0L, 0L, State.TRUNCATE_INVALID_TX, 0L, null, 0L, false, null, null, - truncateInvalidTxTime); + truncateInvalidTxTime, 0L, null); + } + + /** + * Creates a new instance in the {@link State#CHECKPOINT} state. + */ + public static TransactionEdit createCheckpoint(long writePointer, long parentWritePointer) { + return new TransactionEdit(writePointer, 0L, State.CHECKPOINT, 0L, null, 0L, false, null, null, 0L, + parentWritePointer, null); } @Override public void write(DataOutput out) throws IOException { - CODEC_V3.encode(this, out); + CODEC_V4.encode(this, out); } @Override public void readFields(DataInput in) throws IOException { byte version = in.readByte(); switch (version) { + case V4: + CODEC_V4.decode(this, in); + break; case V3: CODEC_V3.decode(this, in); break; @@ -267,13 +304,15 @@ public final boolean equals(Object o) { Objects.equal(this.canCommit, that.canCommit) && Objects.equal(this.type, that.type) && Objects.equal(this.truncateInvalidTx, that.truncateInvalidTx) && - Objects.equal(this.truncateInvalidTxTime, that.truncateInvalidTxTime); + Objects.equal(this.truncateInvalidTxTime, that.truncateInvalidTxTime) && + Objects.equal(this.parentWritePointer, that.parentWritePointer) && + Objects.equal(this.checkpointPointers, that.checkpointPointers); } @Override public final int hashCode() { return Objects.hashCode(writePointer, visibilityUpperBound, commitPointer, expirationDate, state, changes, - canCommit, type); + canCommit, type, parentWritePointer, checkpointPointers); } @Override @@ -289,6 +328,8 @@ public String toString() { .add("type", type) .add("truncateInvalidTx", truncateInvalidTx) .add("truncateInvalidTxTime", truncateInvalidTxTime) + .add("parentWritePointer", parentWritePointer) + .add("checkpointPointers", checkpointPointers) .toString(); } @@ -470,7 +511,7 @@ private Set emptySet(Set set) { @Override public void encode(TransactionEdit src, DataOutput out) throws IOException { - out.writeByte(V3); + writeVersion(out); out.writeLong(src.writePointer); // use ordinal for predictable size, though this does not support evolution out.writeInt(src.state.ordinal()); @@ -505,5 +546,41 @@ public void encode(TransactionEdit src, DataOutput out) throws IOException { } out.writeLong(src.truncateInvalidTxTime); } + + protected void writeVersion(DataOutput out) throws IOException { + out.writeByte(V3); + } + } + + static class TransactionEditCodeV4 extends TransactionEditCodecV3 { + @Override + public void decode(TransactionEdit dest, DataInput in) throws IOException { + super.decode(dest, in); + dest.parentWritePointer = in.readLong(); + int checkpointPointersLen = in.readInt(); + dest.checkpointPointers = new long[checkpointPointersLen]; + for (int i = 0; i < checkpointPointersLen; i++) { + dest.checkpointPointers[i] = in.readLong(); + } + } + + @Override + public void encode(TransactionEdit src, DataOutput out) throws IOException { + super.encode(src, out); + out.writeLong(src.parentWritePointer); + if (src.checkpointPointers == null) { + out.writeInt(0); + } else { + out.writeInt(src.checkpointPointers.length); + for (int i = 0; i < src.checkpointPointers.length; i++) { + out.writeLong(src.checkpointPointers[i]); + } + } + } + + @Override + protected void writeVersion(DataOutput out) throws IOException { + out.writeByte(V4); + } } } diff --git a/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV2.java b/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV2.java index d94b1ccb..0d992910 100644 --- a/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV2.java +++ b/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV2.java @@ -19,6 +19,7 @@ import co.cask.tephra.TransactionManager; import co.cask.tephra.TransactionType; import com.google.common.collect.Maps; +import it.unimi.dsi.fastutil.longs.LongArrayList; import java.io.IOException; import java.util.Map; @@ -69,7 +70,8 @@ protected NavigableMap decodeInProgress(B throw new IOException("Type enum ordinal value is out of range: " + txTypeIdx); } inProgress.put(txId, - new TransactionManager.InProgressTx(visibilityUpperBound, expiration, txType)); + new TransactionManager.InProgressTx(visibilityUpperBound, expiration, txType, + new LongArrayList())); } size = decoder.readInt(); } diff --git a/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV4.java b/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV4.java new file mode 100644 index 00000000..bad4f0da --- /dev/null +++ b/tephra-core/src/main/java/co/cask/tephra/snapshot/SnapshotCodecV4.java @@ -0,0 +1,98 @@ +/* + * Copyright © 2015 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package co.cask.tephra.snapshot; + +import co.cask.tephra.TransactionManager; +import co.cask.tephra.TransactionType; +import com.google.common.collect.Maps; +import it.unimi.dsi.fastutil.longs.LongArrayList; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +/** + * Handles serialization/deserialization of a {@link co.cask.tephra.persist.TransactionSnapshot} + * and its elements to {@code byte[]}. + * + */ +public class SnapshotCodecV4 extends SnapshotCodecV2 { + @Override + public int getVersion() { + return 4; + } + + @Override + protected void encodeInProgress(BinaryEncoder encoder, Map inProgress) + throws IOException { + + if (!inProgress.isEmpty()) { + encoder.writeInt(inProgress.size()); + for (Map.Entry entry : inProgress.entrySet()) { + encoder.writeLong(entry.getKey()); // tx id + encoder.writeLong(entry.getValue().getExpiration()); + encoder.writeLong(entry.getValue().getVisibilityUpperBound()); + encoder.writeInt(entry.getValue().getType().ordinal()); + // write checkpoint tx IDs + LongArrayList checkpointPointers = entry.getValue().getCheckpointWritePointers(); + if (checkpointPointers == null && !checkpointPointers.isEmpty()) { + encoder.writeInt(checkpointPointers.size()); + for (int i = 0; i < checkpointPointers.size(); i++) { + encoder.writeLong(checkpointPointers.getLong(i)); + } + } + encoder.writeInt(0); + } + } + encoder.writeInt(0); // zero denotes end of list as per AVRO spec + } + + @Override + protected NavigableMap decodeInProgress(BinaryDecoder decoder) + throws IOException { + + int size = decoder.readInt(); + NavigableMap inProgress = Maps.newTreeMap(); + while (size != 0) { // zero denotes end of list as per AVRO spec + for (int remaining = size; remaining > 0; --remaining) { + long txId = decoder.readLong(); + long expiration = decoder.readLong(); + long visibilityUpperBound = decoder.readLong(); + int txTypeIdx = decoder.readInt(); + TransactionType txType; + try { + txType = TransactionType.values()[txTypeIdx]; + } catch (ArrayIndexOutOfBoundsException e) { + throw new IOException("Type enum ordinal value is out of range: " + txTypeIdx); + } + // read checkpoint tx IDs + int checkpointPointerSize = decoder.readInt(); + LongArrayList checkpointPointers = new LongArrayList(checkpointPointerSize); + while (checkpointPointerSize != 0) { + for (int checkpointRemaining = checkpointPointerSize; checkpointRemaining > 0; --checkpointRemaining) { + checkpointPointers.add(decoder.readLong()); + } + size = decoder.readInt(); + } + inProgress.put(txId, + new TransactionManager.InProgressTx(visibilityUpperBound, expiration, txType, checkpointPointers)); + } + size = decoder.readInt(); + } + return inProgress; + } +} diff --git a/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java b/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java index 0afa4bf4..3fed8895 100644 --- a/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java +++ b/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java @@ -56,7 +56,8 @@ public static long getOldestVisibleTimestamp(Map ttlByFamily, Tran public static long getMaxVisibleTimestamp(Transaction tx) { // NOTE: +1 here because we want read up to writepointer inclusive, but timerange's end is exclusive // however, we also need to guard against overflow in the case write pointer is set to MAX_VALUE - return tx.getWritePointer() < Long.MAX_VALUE ? tx.getWritePointer() + 1 : tx.getWritePointer(); + return tx.getCurrentWritePointer() < Long.MAX_VALUE ? + tx.getCurrentWritePointer() + 1 : tx.getCurrentWritePointer(); } /** diff --git a/tephra-core/src/main/thrift/transaction.thrift b/tephra-core/src/main/thrift/transaction.thrift index 559cbfeb..5d92b88e 100644 --- a/tephra-core/src/main/thrift/transaction.thrift +++ b/tephra-core/src/main/thrift/transaction.thrift @@ -26,6 +26,8 @@ struct TTransaction { 4: list inProgress, 5: i64 firstShort, 6: TTransactionType type, + 7: i64 currentWritePointer, + 8: list checkpointWritePointers, } exception TTransactionNotInProgressException { @@ -60,4 +62,5 @@ service TTransactionServer { TBoolean truncateInvalidTx(1: set txns), TBoolean truncateInvalidTxBefore(1: i64 time) throws (1: TInvalidTruncateTimeException e), i32 invalidTxSize(), + TTransaction checkpoint(1: TTransaction tx) throws (1: TTransactionNotInProgressException e), } diff --git a/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java b/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java index 5259f0eb..8ae04edf 100644 --- a/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java @@ -367,7 +367,7 @@ public void testLongTxnEditReplay() throws Exception { long time1 = System.currentTimeMillis(); long wp1 = time1 * TxConstants.MAX_TX_PER_MS; TransactionEdit edit1 = TransactionEdit.createStarted(wp1, wp1 - 10, time1 + 100000, TransactionType.LONG); - TransactionEdit edit2 = TransactionEdit.createAborted(wp1, TransactionType.LONG); + TransactionEdit edit2 = TransactionEdit.createAborted(wp1, TransactionType.LONG, null); long time2 = time1 + 100; long wp2 = time2 * TxConstants.MAX_TX_PER_MS; @@ -378,7 +378,7 @@ public void testLongTxnEditReplay() throws Exception { long wp3 = time3 * TxConstants.MAX_TX_PER_MS; TransactionEdit edit5 = TransactionEdit.createStarted(wp3, wp3 - 10, time3 + 100000, TransactionType.LONG); TransactionEdit edit6 = TransactionEdit.createInvalid(wp3); - TransactionEdit edit7 = TransactionEdit.createAborted(wp3, TransactionType.LONG); + TransactionEdit edit7 = TransactionEdit.createAborted(wp3, TransactionType.LONG, null); // write transaction edits TransactionLog log = storage.createLog(time1); @@ -580,7 +580,7 @@ private List createRandomEdits(int numEntries) { edits.add(TransactionEdit.createInvalid(writePointer)); break; case ABORTED: - edits.add(TransactionEdit.createAborted(writePointer, TransactionType.SHORT)); + edits.add(TransactionEdit.createAborted(writePointer, TransactionType.SHORT, null)); break; case MOVE_WATERMARK: edits.add(TransactionEdit.createMoveWatermark(writePointer)); diff --git a/tephra-core/src/test/java/co/cask/tephra/persist/LocalTransactionStateStorageTest.java b/tephra-core/src/test/java/co/cask/tephra/persist/LocalTransactionStateStorageTest.java index 5ae2eea2..c95a2601 100644 --- a/tephra-core/src/test/java/co/cask/tephra/persist/LocalTransactionStateStorageTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/persist/LocalTransactionStateStorageTest.java @@ -72,7 +72,7 @@ private class TransactionEditV2 extends TransactionEdit { public TransactionEditV2(long writePointer, long visibilityUpperBound, State state, long expirationDate, Set changes, long commitPointer, boolean canCommit, TransactionType type) { super(writePointer, visibilityUpperBound, state, expirationDate, changes, commitPointer, canCommit, type, - null, 0L); + null, 0L, 0L, null); } @Override public void write(DataOutput out) throws IOException { diff --git a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java index adeb4931..a123c3f3 100644 --- a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java +++ b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -124,31 +123,38 @@ protected boolean doCommit() throws IOException { @Override protected boolean doRollback() throws Exception { try { - List rollbackDeletes = new ArrayList(changeSet.size()); - for (ActionChange change : changeSet) { - byte[] row = change.getRow(); - byte[] family = change.getFamily(); - byte[] qualifier = change.getQualifier(); - long transactionTimestamp = tx.getWritePointer(); - Delete rollbackDelete = new Delete(row); - rollbackDelete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]); - switch (conflictLevel) { - case ROW: - case NONE: - // issue family delete for the tx write pointer - rollbackDelete.deleteFamilyVersion(change.getFamily(), transactionTimestamp); - break; - case COLUMN: - if (family != null && qualifier == null) { - rollbackDelete.deleteFamilyVersion(family, transactionTimestamp); - } else if (family != null && qualifier != null) { - rollbackDelete.deleteColumn(family, qualifier, transactionTimestamp); - } - break; - default: - throw new IllegalStateException("Unknown conflict detection level: " + conflictLevel); + // pre-size arraylist of deletes + int size = 0; + for (Set cs : changeSets.values()) { + size += cs.size(); + } + List rollbackDeletes = new ArrayList(size); + for (Map.Entry> entry : changeSets.entrySet()) { + long transactionTimestamp = entry.getKey(); + for (ActionChange change : entry.getValue()) { + byte[] row = change.getRow(); + byte[] family = change.getFamily(); + byte[] qualifier = change.getQualifier(); + Delete rollbackDelete = new Delete(row); + rollbackDelete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]); + switch (conflictLevel) { + case ROW: + case NONE: + // issue family delete for the tx write pointer + rollbackDelete.deleteFamilyVersion(change.getFamily(), transactionTimestamp); + break; + case COLUMN: + if (family != null && qualifier == null) { + rollbackDelete.deleteFamilyVersion(family, transactionTimestamp); + } else if (family != null && qualifier != null) { + rollbackDelete.deleteColumn(family, qualifier, transactionTimestamp); + } + break; + default: + throw new IllegalStateException("Unknown conflict detection level: " + conflictLevel); + } + rollbackDeletes.add(rollbackDelete); } - rollbackDeletes.add(rollbackDelete); } hTable.delete(rollbackDeletes); return true; @@ -159,7 +165,7 @@ protected boolean doRollback() throws Exception { LOG.error("Could not flush HTable commits", e); } tx = null; - changeSet.clear(); + changeSets.clear(); } } @@ -493,14 +499,14 @@ private Scan transactionalizeAction(Scan scan) throws IOException { } private Put transactionalizeAction(Put put) throws IOException { - Put txPut = new Put(put.getRow(), tx.getWritePointer()); + Put txPut = new Put(put.getRow(), tx.getCurrentWritePointer()); Set>> familyMap = put.getFamilyCellMap().entrySet(); if (!familyMap.isEmpty()) { for (Map.Entry> family : familyMap) { List familyValues = family.getValue(); if (!familyValues.isEmpty()) { for (Cell value : familyValues) { - txPut.add(value.getFamily(), value.getQualifier(), tx.getWritePointer(), value.getValue()); + txPut.add(value.getFamily(), value.getQualifier(), tx.getCurrentWritePointer(), value.getValue()); addToChangeSet(txPut.getRow(), value.getFamily(), value.getQualifier()); } } @@ -515,7 +521,7 @@ private Put transactionalizeAction(Put put) throws IOException { } private Delete transactionalizeAction(Delete delete) throws IOException { - long transactionTimestamp = tx.getWritePointer(); + long transactionTimestamp = tx.getCurrentWritePointer(); byte[] deleteRow = delete.getRow(); Delete txDelete = new Delete(deleteRow, transactionTimestamp); diff --git a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java index f3b3990b..ea820130 100644 --- a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java +++ b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java @@ -149,10 +149,13 @@ public void preGetOp(ObserverContext e, Get get, L throws IOException { Transaction tx = getFromOperation(get); if (tx != null) { + boolean excludeCurrentWritePtr = isAttributeSet(get, TxConstants.TX_EXCLUDE_CURRENT_WRITE); + projectFamilyDeletes(get); get.setMaxVersions(); get.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx), TxUtils.getMaxVisibleTimestamp(tx)); - Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN), get.getFilter()); + Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN, excludeCurrentWritePtr), + get.getFilter()); get.setFilter(newFilter); } } @@ -200,10 +203,13 @@ public RegionScanner preScannerOpen(ObserverContext ttlByFamily */ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, ScanType scanType, @Nullable Filter cellFilter) { + this(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter, false); + } + + /** + * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. + * + * @param tx the current transaction to apply. Only data visible to this transaction will be returned. + * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name + * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} + * these will be interpreted as "delete" markers and the column will be filtered out + * @param scanType the type of scan operation being performed + * @param cellFilter if non-null, this filter will be applied to all cells visible to the current transaction, by + * calling {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}. If null, then + * {@link Filter.ReturnCode#INCLUDE_AND_NEXT_COL} will be returned instead. + * @param excludeCurrentWritePtr if true, cells written with the transaction's current write pointer will be filtered + * out from results returned by this filter. + */ + public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, + ScanType scanType, @Nullable Filter cellFilter, boolean excludeCurrentWritePtr) { this.tx = tx; this.oldestTsByFamily = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); for (Map.Entry ttlEntry : ttlByFamily.entrySet()) { @@ -93,6 +114,7 @@ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily this.clearDeletes = scanType == ScanType.COMPACT_DROP_DELETES || scanType == ScanType.USER_SCAN; this.cellFilter = cellFilter; + this.excludeCurrentWritePtr = excludeCurrentWritePtr; } @Override @@ -109,7 +131,7 @@ public ReturnCode filterKeyValue(Cell cell) throws IOException { if (kvTimestamp < currentOldestTs) { // passed TTL for this column, seek to next return ReturnCode.NEXT_COL; - } else if (tx.isVisible(kvTimestamp)) { + } else if (tx.isVisible(kvTimestamp, excludeCurrentWritePtr)) { if (deleteTracker.isFamilyDelete(cell)) { deleteTracker.addFamilyDelete(cell); if (clearDeletes) { diff --git a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java index cfdd11df..228c7053 100644 --- a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java +++ b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java @@ -19,6 +19,7 @@ import co.cask.tephra.TransactionConflictException; import co.cask.tephra.TransactionContext; import co.cask.tephra.TransactionManager; +import co.cask.tephra.TransactionSystemClient; import co.cask.tephra.TxConstants; import co.cask.tephra.hbase96.coprocessor.TransactionProcessor; import co.cask.tephra.inmemory.InMemoryTxSystemClient; @@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -137,6 +139,7 @@ private HTable createTable(byte[] tableName, byte[][] columnFamilies) throws Exc /** * Test transactional put and get requests. + * * @throws Exception */ @Test @@ -157,6 +160,7 @@ public void testValidTransactionalPutAndGet() throws Exception { /** * Test aborted put requests, that must be rolled back. + * * @throws Exception */ @Test @@ -177,6 +181,7 @@ public void testAbortedTransactionPutAndGet() throws Exception { /** * Test transactional delete operations. + * * @throws Exception */ @Test @@ -321,6 +326,7 @@ public void testValidTransactionalDelete() throws Exception { /** * Test aborted transactional delete requests, that must be rolled back. + * * @throws Exception */ @Test @@ -516,6 +522,7 @@ public void testRowDelete() throws Exception { /** * Expect an exception since a transaction hasn't been started. + * * @throws Exception */ @Test(expected = IOException.class) @@ -751,4 +758,166 @@ public void testNoneLevelConflictDetection() throws Exception { assertTrue(row3.isEmpty()); txContext2.finish(); } + + @Test + public void testCheckpoint() throws Exception { + // start a transaction, using checkpoints between writes + transactionContext.start(); + transactionAwareHTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + Transaction origTx = transactionContext.getCurrentTransaction(); + transactionContext.checkpoint(); + Transaction postCheckpointTx = transactionContext.getCurrentTransaction(); + + assertEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer()); + assertNotEquals(origTx.getCurrentWritePointer(), postCheckpointTx.getCurrentWritePointer()); + long[] checkpointPtrs = postCheckpointTx.getCheckpointWritePointers(); + assertEquals(1, checkpointPtrs.length); + assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs[0]); + + transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); + transactionContext.checkpoint(); + Transaction postCheckpointTx2 = transactionContext.getCurrentTransaction(); + + assertEquals(origTx.getWritePointer(), postCheckpointTx2.getWritePointer()); + assertNotEquals(postCheckpointTx.getCurrentWritePointer(), postCheckpointTx2.getCurrentWritePointer()); + long[] checkpointPtrs2 = postCheckpointTx2.getCheckpointWritePointers(); + assertEquals(2, checkpointPtrs2.length); + assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs2[0]); + assertEquals(postCheckpointTx2.getCurrentWritePointer(), checkpointPtrs2[1]); + + transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + + // by default, all rows should be visible with Read-Your-Writes + verifyRow(transactionAwareHTable, TestBytes.row, TestBytes.value); + verifyRow(transactionAwareHTable, TestBytes.row2, TestBytes.value2); + verifyRow(transactionAwareHTable, TestBytes.row3, TestBytes.value); + + // when disabling current write pointer, only the previous checkpoints should be visible + Get get = new Get(TestBytes.row); + get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + verifyRow(transactionAwareHTable, get, TestBytes.value); + get = new Get(TestBytes.row2); + get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + verifyRow(transactionAwareHTable, get, TestBytes.value2); + get = new Get(TestBytes.row3); + get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + verifyRow(transactionAwareHTable, get, null); + + // test scan results excluding current write pointer + Scan scan = new Scan(); + scan.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + ResultScanner scanner = transactionAwareHTable.getScanner(scan); + + Result row = scanner.next(); + assertNotNull(row); + assertArrayEquals(TestBytes.row, row.getRow()); + assertEquals(1, row.size()); + assertArrayEquals(TestBytes.value, row.getValue(TestBytes.family, TestBytes.qualifier)); + + row = scanner.next(); + assertNotNull(row); + assertArrayEquals(TestBytes.row2, row.getRow()); + assertEquals(1, row.size()); + assertArrayEquals(TestBytes.value2, row.getValue(TestBytes.family, TestBytes.qualifier)); + + row = scanner.next(); + assertNull(row); + scanner.close(); + + // check that writes are still not visible to other clients + TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); + TransactionContext txContext2 = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable2); + + txContext2.start(); + verifyRow(txTable2, TestBytes.row, null); + verifyRow(txTable2, TestBytes.row2, null); + verifyRow(txTable2, TestBytes.row3, null); + txContext2.finish(); + + // commit transaction, verify writes are visible + transactionContext.finish(); + + txContext2.start(); + verifyRow(txTable2, TestBytes.row, TestBytes.value); + verifyRow(txTable2, TestBytes.row2, TestBytes.value2); + verifyRow(txTable2, TestBytes.row3, TestBytes.value); + txContext2.finish(); + txTable2.close(); + } + + @Test + public void testCheckpointRollback() throws Exception { + // start a transaction, using checkpoints between writes + transactionContext.start(); + transactionAwareHTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + transactionContext.checkpoint(); + transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); + transactionContext.checkpoint(); + transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + + transactionContext.abort(); + + transactionContext.start(); + verifyRow(transactionAwareHTable, TestBytes.row, null); + verifyRow(transactionAwareHTable, TestBytes.row2, null); + verifyRow(transactionAwareHTable, TestBytes.row3, null); + + Scan scan = new Scan(); + ResultScanner scanner = transactionAwareHTable.getScanner(scan); + assertNull(scanner.next()); + scanner.close(); + transactionContext.finish(); + } + + @Test + public void testCheckpointInvalidate() throws Exception { + // start a transaction, using checkpoints between writes + transactionContext.start(); + Transaction origTx = transactionContext.getCurrentTransaction(); + transactionAwareHTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + transactionContext.checkpoint(); + Transaction checkpointTx1 = transactionContext.getCurrentTransaction(); + transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); + transactionContext.checkpoint(); + Transaction checkpointTx2 = transactionContext.getCurrentTransaction(); + transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + + TransactionSystemClient txClient = new InMemoryTxSystemClient(txManager); + txClient.invalidate(transactionContext.getCurrentTransaction().getWritePointer()); + + // check that writes are not visible + TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); + TransactionContext txContext2 = new TransactionContext(txClient, txTable2); + txContext2.start(); + Transaction newTx = txContext2.getCurrentTransaction(); + + // all 3 writes pointers from the previous transaction should now be excluded + assertTrue(newTx.isExcluded(origTx.getCurrentWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx1.getCurrentWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx2.getCurrentWritePointer())); + + verifyRow(txTable2, TestBytes.row, null); + verifyRow(txTable2, TestBytes.row2, null); + verifyRow(txTable2, TestBytes.row3, null); + + Scan scan = new Scan(); + ResultScanner scanner = txTable2.getScanner(scan); + assertNull(scanner.next()); + scanner.close(); + txContext2.finish(); + } + + private void verifyRow(HTableInterface table, byte[] rowkey, byte[] expectedValue) throws Exception { + verifyRow(table, new Get(rowkey), expectedValue); + } + + private void verifyRow(HTableInterface table, Get get, byte[] expectedValue) throws Exception { + Result result = table.get(get); + if (expectedValue == null) { + assertTrue(result.isEmpty()); + } else { + assertFalse(result.isEmpty()); + assertArrayEquals(expectedValue, result.getValue(TestBytes.family, TestBytes.qualifier)); + } + } } diff --git a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java index 02ecf3ee..e2d480d8 100644 --- a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java +++ b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java @@ -127,31 +127,38 @@ protected boolean doCommit() throws IOException { @Override protected boolean doRollback() throws Exception { try { - List rollbackDeletes = new ArrayList(changeSet.size()); - for (ActionChange change : changeSet) { - byte[] row = change.getRow(); - byte[] family = change.getFamily(); - byte[] qualifier = change.getQualifier(); - long transactionTimestamp = tx.getWritePointer(); - Delete rollbackDelete = new Delete(row); - rollbackDelete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]); - switch (conflictLevel) { - case ROW: - case NONE: - // issue family delete for the tx write pointer - rollbackDelete.deleteFamilyVersion(change.getFamily(), transactionTimestamp); - break; - case COLUMN: - if (family != null && qualifier == null) { - rollbackDelete.deleteFamilyVersion(family, transactionTimestamp); - } else if (family != null && qualifier != null) { - rollbackDelete.deleteColumn(family, qualifier, transactionTimestamp); - } - break; - default: - throw new IllegalStateException("Unknown conflict detection level: " + conflictLevel); + // pre-size arraylist of deletes + int size = 0; + for (Set cs : changeSets.values()) { + size += cs.size(); + } + List rollbackDeletes = new ArrayList(size); + for (Map.Entry> entry : changeSets.entrySet()) { + long transactionTimestamp = entry.getKey(); + for (ActionChange change : entry.getValue()) { + byte[] row = change.getRow(); + byte[] family = change.getFamily(); + byte[] qualifier = change.getQualifier(); + Delete rollbackDelete = new Delete(row); + rollbackDelete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]); + switch (conflictLevel) { + case ROW: + case NONE: + // issue family delete for the tx write pointer + rollbackDelete.deleteFamilyVersion(change.getFamily(), transactionTimestamp); + break; + case COLUMN: + if (family != null && qualifier == null) { + rollbackDelete.deleteFamilyVersion(family, transactionTimestamp); + } else if (family != null && qualifier != null) { + rollbackDelete.deleteColumn(family, qualifier, transactionTimestamp); + } + break; + default: + throw new IllegalStateException("Unknown conflict detection level: " + conflictLevel); + } + rollbackDeletes.add(rollbackDelete); } - rollbackDeletes.add(rollbackDelete); } hTable.delete(rollbackDeletes); return true; @@ -162,7 +169,7 @@ protected boolean doRollback() throws Exception { LOG.error("Could not flush HTable commits", e); } tx = null; - changeSet.clear(); + changeSets.clear(); } } @@ -521,14 +528,14 @@ private Scan transactionalizeAction(Scan scan) throws IOException { } private Put transactionalizeAction(Put put) throws IOException { - Put txPut = new Put(put.getRow(), tx.getWritePointer()); + Put txPut = new Put(put.getRow(), tx.getCurrentWritePointer()); Set>> familyMap = put.getFamilyCellMap().entrySet(); if (!familyMap.isEmpty()) { for (Map.Entry> family : familyMap) { List familyValues = family.getValue(); if (!familyValues.isEmpty()) { for (Cell value : familyValues) { - txPut.add(value.getFamily(), value.getQualifier(), tx.getWritePointer(), value.getValue()); + txPut.add(value.getFamily(), value.getQualifier(), tx.getCurrentWritePointer(), value.getValue()); addToChangeSet(txPut.getRow(), value.getFamily(), value.getQualifier()); } } @@ -543,7 +550,7 @@ private Put transactionalizeAction(Put put) throws IOException { } private Delete transactionalizeAction(Delete delete) throws IOException { - long transactionTimestamp = tx.getWritePointer(); + long transactionTimestamp = tx.getCurrentWritePointer(); byte[] deleteRow = delete.getRow(); Delete txDelete = new Delete(deleteRow, transactionTimestamp); diff --git a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java index 8478141f..bae143a3 100644 --- a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java +++ b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java @@ -149,10 +149,13 @@ public void preGetOp(ObserverContext e, Get get, L throws IOException { Transaction tx = getFromOperation(get); if (tx != null) { + boolean excludeCurrentWritePtr = isAttributeSet(get, TxConstants.TX_EXCLUDE_CURRENT_WRITE); + projectFamilyDeletes(get); get.setMaxVersions(); get.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx), TxUtils.getMaxVisibleTimestamp(tx)); - Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN), get.getFilter()); + Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN, excludeCurrentWritePtr), + get.getFilter()); get.setFilter(newFilter); } } @@ -200,10 +203,13 @@ public RegionScanner preScannerOpen(ObserverContext ttlByFamily */ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, ScanType scanType, @Nullable Filter cellFilter) { + this(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter, false); + } + + /** + * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. + * + * @param tx the current transaction to apply. Only data visible to this transaction will be returned. + * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name + * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} + * these will be interpreted as "delete" markers and the column will be filtered out + * @param scanType the type of scan operation being performed + * @param cellFilter if non-null, this filter will be applied to all cells visible to the current transaction, by + * calling {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}. If null, then + * {@link Filter.ReturnCode#INCLUDE_AND_NEXT_COL} will be returned instead. + * @param excludeCurrentWritePtr if true, cells written with the transaction's current write pointer will be filtered + * out from results returned by this filter. + */ + public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, + ScanType scanType, @Nullable Filter cellFilter, boolean excludeCurrentWritePtr) { this.tx = tx; this.oldestTsByFamily = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); for (Map.Entry ttlEntry : ttlByFamily.entrySet()) { @@ -93,6 +114,7 @@ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily this.clearDeletes = scanType == ScanType.COMPACT_DROP_DELETES || scanType == ScanType.USER_SCAN; this.cellFilter = cellFilter; + this.excludeCurrentWritePtr = excludeCurrentWritePtr; } @Override @@ -109,7 +131,7 @@ public ReturnCode filterKeyValue(Cell cell) throws IOException { if (kvTimestamp < currentOldestTs) { // passed TTL for this column, seek to next return ReturnCode.NEXT_COL; - } else if (tx.isVisible(kvTimestamp)) { + } else if (tx.isVisible(kvTimestamp, excludeCurrentWritePtr)) { if (deleteTracker.isFamilyDelete(cell)) { deleteTracker.addFamilyDelete(cell); if (clearDeletes) { diff --git a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java index a01883e9..18decde0 100644 --- a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java +++ b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java @@ -19,6 +19,7 @@ import co.cask.tephra.TransactionConflictException; import co.cask.tephra.TransactionContext; import co.cask.tephra.TransactionManager; +import co.cask.tephra.TransactionSystemClient; import co.cask.tephra.TxConstants; import co.cask.tephra.hbase98.coprocessor.TransactionProcessor; import co.cask.tephra.inmemory.InMemoryTxSystemClient; @@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -756,4 +758,166 @@ public void testNoneLevelConflictDetection() throws Exception { assertTrue(row3.isEmpty()); txContext2.finish(); } + + @Test + public void testCheckpoint() throws Exception { + // start a transaction, using checkpoints between writes + transactionContext.start(); + transactionAwareHTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + Transaction origTx = transactionContext.getCurrentTransaction(); + transactionContext.checkpoint(); + Transaction postCheckpointTx = transactionContext.getCurrentTransaction(); + + assertEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer()); + assertNotEquals(origTx.getCurrentWritePointer(), postCheckpointTx.getCurrentWritePointer()); + long[] checkpointPtrs = postCheckpointTx.getCheckpointWritePointers(); + assertEquals(1, checkpointPtrs.length); + assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs[0]); + + transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); + transactionContext.checkpoint(); + Transaction postCheckpointTx2 = transactionContext.getCurrentTransaction(); + + assertEquals(origTx.getWritePointer(), postCheckpointTx2.getWritePointer()); + assertNotEquals(postCheckpointTx.getCurrentWritePointer(), postCheckpointTx2.getCurrentWritePointer()); + long[] checkpointPtrs2 = postCheckpointTx2.getCheckpointWritePointers(); + assertEquals(2, checkpointPtrs2.length); + assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs2[0]); + assertEquals(postCheckpointTx2.getCurrentWritePointer(), checkpointPtrs2[1]); + + transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + + // by default, all rows should be visible with Read-Your-Writes + verifyRow(transactionAwareHTable, TestBytes.row, TestBytes.value); + verifyRow(transactionAwareHTable, TestBytes.row2, TestBytes.value2); + verifyRow(transactionAwareHTable, TestBytes.row3, TestBytes.value); + + // when disabling current write pointer, only the previous checkpoints should be visible + Get get = new Get(TestBytes.row); + get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + verifyRow(transactionAwareHTable, get, TestBytes.value); + get = new Get(TestBytes.row2); + get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + verifyRow(transactionAwareHTable, get, TestBytes.value2); + get = new Get(TestBytes.row3); + get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + verifyRow(transactionAwareHTable, get, null); + + // test scan results excluding current write pointer + Scan scan = new Scan(); + scan.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); + ResultScanner scanner = transactionAwareHTable.getScanner(scan); + + Result row = scanner.next(); + assertNotNull(row); + assertArrayEquals(TestBytes.row, row.getRow()); + assertEquals(1, row.size()); + assertArrayEquals(TestBytes.value, row.getValue(TestBytes.family, TestBytes.qualifier)); + + row = scanner.next(); + assertNotNull(row); + assertArrayEquals(TestBytes.row2, row.getRow()); + assertEquals(1, row.size()); + assertArrayEquals(TestBytes.value2, row.getValue(TestBytes.family, TestBytes.qualifier)); + + row = scanner.next(); + assertNull(row); + scanner.close(); + + // check that writes are still not visible to other clients + TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); + TransactionContext txContext2 = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable2); + + txContext2.start(); + verifyRow(txTable2, TestBytes.row, null); + verifyRow(txTable2, TestBytes.row2, null); + verifyRow(txTable2, TestBytes.row3, null); + txContext2.finish(); + + // commit transaction, verify writes are visible + transactionContext.finish(); + + txContext2.start(); + verifyRow(txTable2, TestBytes.row, TestBytes.value); + verifyRow(txTable2, TestBytes.row2, TestBytes.value2); + verifyRow(txTable2, TestBytes.row3, TestBytes.value); + txContext2.finish(); + txTable2.close(); + } + + @Test + public void testCheckpointRollback() throws Exception { + // start a transaction, using checkpoints between writes + transactionContext.start(); + transactionAwareHTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + transactionContext.checkpoint(); + transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); + transactionContext.checkpoint(); + transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + + transactionContext.abort(); + + transactionContext.start(); + verifyRow(transactionAwareHTable, TestBytes.row, null); + verifyRow(transactionAwareHTable, TestBytes.row2, null); + verifyRow(transactionAwareHTable, TestBytes.row3, null); + + Scan scan = new Scan(); + ResultScanner scanner = transactionAwareHTable.getScanner(scan); + assertNull(scanner.next()); + scanner.close(); + transactionContext.finish(); + } + + @Test + public void testCheckpointInvalidate() throws Exception { + // start a transaction, using checkpoints between writes + transactionContext.start(); + Transaction origTx = transactionContext.getCurrentTransaction(); + transactionAwareHTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + transactionContext.checkpoint(); + Transaction checkpointTx1 = transactionContext.getCurrentTransaction(); + transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); + transactionContext.checkpoint(); + Transaction checkpointTx2 = transactionContext.getCurrentTransaction(); + transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); + + TransactionSystemClient txClient = new InMemoryTxSystemClient(txManager); + txClient.invalidate(transactionContext.getCurrentTransaction().getWritePointer()); + + // check that writes are not visible + TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); + TransactionContext txContext2 = new TransactionContext(txClient, txTable2); + txContext2.start(); + Transaction newTx = txContext2.getCurrentTransaction(); + + // all 3 writes pointers from the previous transaction should now be excluded + assertTrue(newTx.isExcluded(origTx.getCurrentWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx1.getCurrentWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx2.getCurrentWritePointer())); + + verifyRow(txTable2, TestBytes.row, null); + verifyRow(txTable2, TestBytes.row2, null); + verifyRow(txTable2, TestBytes.row3, null); + + Scan scan = new Scan(); + ResultScanner scanner = txTable2.getScanner(scan); + assertNull(scanner.next()); + scanner.close(); + txContext2.finish(); + } + + private void verifyRow(HTableInterface table, byte[] rowkey, byte[] expectedValue) throws Exception { + verifyRow(table, new Get(rowkey), expectedValue); + } + + private void verifyRow(HTableInterface table, Get get, byte[] expectedValue) throws Exception { + Result result = table.get(get); + if (expectedValue == null) { + assertTrue(result.isEmpty()); + } else { + assertFalse(result.isEmpty()); + assertArrayEquals(expectedValue, result.getValue(TestBytes.family, TestBytes.qualifier)); + } + } } From b4302d6d99f7d920858959f959b342cc7a8b52a6 Mon Sep 17 00:00:00 2001 From: Gary Helmling Date: Wed, 20 May 2015 15:30:04 -0700 Subject: [PATCH 2/5] TEPHRA-98 Separate transaction ID from write pointer --- .../main/java/co/cask/tephra/Transaction.java | 48 +++++++-------- .../tephra/AbstractTransactionAwareTable.java | 2 +- .../co/cask/tephra/TransactionContext.java | 28 ++++----- .../co/cask/tephra/TransactionManager.java | 38 ++++++------ .../TransactionConverterUtils.java | 4 +- .../distributed/TransactionServiceClient.java | 2 +- .../java/co/cask/tephra/util/TxUtils.java | 4 +- .../co/cask/tephra/TransactionAdminTest.java | 12 ++-- .../cask/tephra/TransactionManagerTest.java | 58 ++++++++++--------- .../co/cask/tephra/TransactionSystemTest.java | 34 +++++------ .../AbstractTransactionStateStorageTest.java | 18 +++--- .../hbase96/TransactionAwareHTable.java | 6 +- .../hbase96/TransactionAwareHTableTest.java | 23 ++++---- .../TransactionVisibilityFilterTest.java | 12 ++-- .../hbase98/TransactionAwareHTable.java | 6 +- .../hbase98/TransactionAwareHTableTest.java | 22 +++---- .../TransactionVisibilityFilterTest.java | 12 ++-- 17 files changed, 165 insertions(+), 164 deletions(-) diff --git a/tephra-api/src/main/java/co/cask/tephra/Transaction.java b/tephra-api/src/main/java/co/cask/tephra/Transaction.java index 617b538f..c7315f49 100644 --- a/tephra-api/src/main/java/co/cask/tephra/Transaction.java +++ b/tephra-api/src/main/java/co/cask/tephra/Transaction.java @@ -25,8 +25,8 @@ // are available public class Transaction { private final long readPointer; + private final long txId; private final long writePointer; - private final long currentWritePointer; private final long[] invalids; private final long[] inProgress; private final long firstShortInProgress; @@ -42,36 +42,36 @@ public class Transaction { /** * Creates a new short transaction. * @param readPointer read pointer for transaction - * @param writePointer write pointer for transaction. This uniquely identifies the transaction. + * @param txId unique identifier for the transaction * @param invalids list of invalid transactions to exclude while reading * @param inProgress list of in-progress transactions to exclude while reading * @param firstShortInProgress earliest in-progress short transaction */ - public Transaction(long readPointer, long writePointer, long[] invalids, long[] inProgress, + public Transaction(long readPointer, long txId, long[] invalids, long[] inProgress, long firstShortInProgress) { - this(readPointer, writePointer, invalids, inProgress, firstShortInProgress, TransactionType.SHORT); + this(readPointer, txId, invalids, inProgress, firstShortInProgress, TransactionType.SHORT); } /** * Creates a new transaction. * @param readPointer read pointer for transaction - * @param writePointer write pointer for transaction. This uniquely identifies the transaction. + * @param txId unique identifier for the transaction * @param invalids list of invalid transactions to exclude while reading * @param inProgress list of in-progress transactions to exclude while reading * @param firstShortInProgress earliest in-progress short transaction * @param type transaction type */ - public Transaction(long readPointer, long writePointer, long[] invalids, long[] inProgress, + public Transaction(long readPointer, long txId, long[] invalids, long[] inProgress, long firstShortInProgress, TransactionType type) { - this(readPointer, writePointer, writePointer, invalids, inProgress, firstShortInProgress, type, new long[0]); + this(readPointer, txId, txId, invalids, inProgress, firstShortInProgress, type, new long[0]); } /** * Creates a new transaction. * @param readPointer read pointer for transaction - * @param writePointer write pointer for transaction. This uniquely identifies the transaction. - * @param currentWritePointer the current pointer to be used for any writes. - * For new transactions, this will be the same as {@code writePointer}. For checkpointed + * @param txId unique identifier for the transaction + * @param writePointer the current pointer to be used for any writes. + * For new transactions, this will be the same as {@code txId}. For checkpointed * transactions, this will be the most recent write pointer issued. * @param invalids list of invalid transactions to exclude while reading * @param inProgress list of in-progress transactions to exclude while reading @@ -79,11 +79,11 @@ public Transaction(long readPointer, long writePointer, long[] invalids, long[] * @param type transaction type * @param checkpointPointers the list of writer pointers added from checkpoints on the transaction */ - public Transaction(long readPointer, long writePointer, long currentWritePointer, long[] invalids, long[] inProgress, + public Transaction(long readPointer, long txId, long writePointer, long[] invalids, long[] inProgress, long firstShortInProgress, TransactionType type, long[] checkpointPointers) { this.readPointer = readPointer; + this.txId = txId; this.writePointer = writePointer; - this.currentWritePointer = currentWritePointer; this.invalids = invalids; this.inProgress = inProgress; this.firstShortInProgress = firstShortInProgress; @@ -96,11 +96,11 @@ public Transaction(long readPointer, long writePointer, long currentWritePointer * with the updated checkpoint write pointers. * * @param toCopy the original transaction containing the state to copy - * @param currentWritePointer the new write pointer to use for the transaction + * @param writePointer the new write pointer to use for the transaction * @param checkpointPointers the list of write pointers added from checkpoints on the transaction */ - public Transaction(Transaction toCopy, long currentWritePointer, long[] checkpointPointers) { - this(toCopy.getReadPointer(), toCopy.getWritePointer(), currentWritePointer, toCopy.getInvalids(), + public Transaction(Transaction toCopy, long writePointer, long[] checkpointPointers) { + this(toCopy.getReadPointer(), toCopy.getTransactionId(), writePointer, toCopy.getInvalids(), toCopy.getInProgress(), toCopy.getFirstShortInProgress(), toCopy.getType(), checkpointPointers); } @@ -113,17 +113,17 @@ public long getReadPointer() { * transaction, and uniquely identifies it with the transaction service. This value should be provided * to identify the transaction when calling any transaction lifecycle methods on the transaction service. */ - public long getWritePointer() { - return writePointer; + public long getTransactionId() { + return txId; } /** * Returns the write pointer to be used in persisting any changes. After a checkpoint is performed, this will differ - * from {@link #getWritePointer()}. This method should always be used when setting the timestamp for writes + * from {@link #getTransactionId()}. This method should always be used when setting the timestamp for writes * in order to ensure that the correct value is used. */ - public long getCurrentWritePointer() { - return currentWritePointer; + public long getWritePointer() { + return writePointer; } public long[] getInvalids() { @@ -150,7 +150,7 @@ public long getVisibilityUpperBound() { // NOTE: in some cases when we do not provide visibility guarantee, we set readPointer to MAX value, but // at same time we don't want that to case cleanup everything as this is used for tx janitor + ttl to see // what can be cleaned up. When non-tx mode is implemented better, we should not need this check - return inProgress.length == 0 ? Math.min(writePointer - 1, readPointer) : inProgress[0] - 1; + return inProgress.length == 0 ? Math.min(txId - 1, readPointer) : inProgress[0] - 1; } public long getFirstShortInProgress() { @@ -209,8 +209,8 @@ public boolean isVisible(long version) { public boolean isVisible(long version, boolean excludeCurrentWritePointer) { // either it was committed before or the change belongs to current tx return (version <= getReadPointer() && !isExcluded(version)) || - ((writePointer == version || isCheckpoint(version)) && - (!excludeCurrentWritePointer || currentWritePointer != version)); + ((txId == version || isCheckpoint(version)) && + (!excludeCurrentWritePointer || writePointer != version)); } public boolean hasExcludes() { @@ -237,8 +237,8 @@ public String toString() { .append(Transaction.class.getSimpleName()) .append('{') .append("readPointer: ").append(readPointer) + .append(", transactionId: ").append(txId) .append(", writePointer: ").append(writePointer) - .append(", currentWritePointer: ").append(currentWritePointer) .append(", invalids: ").append(Arrays.toString(invalids)) .append(", inProgress: ").append(Arrays.toString(inProgress)) .append(", firstShortInProgress: ").append(firstShortInProgress) diff --git a/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java b/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java index 73d1d0b1..38da0115 100644 --- a/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java +++ b/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java @@ -142,7 +142,7 @@ public boolean rollbackTx() throws Exception { protected abstract boolean doRollback() throws Exception; protected void addToChangeSet(byte[] row, byte[] family, byte[] qualifier) { - long currentWritePointer = tx.getCurrentWritePointer(); + long currentWritePointer = tx.getWritePointer(); Set changeSet = changeSets.get(currentWritePointer); if (changeSet == null) { changeSet = Sets.newHashSet(); diff --git a/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java b/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java index 86e74d05..2d96061d 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java +++ b/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java @@ -74,7 +74,7 @@ public void start() throws TransactionFailureException { txAware.startTx(currentTx); } catch (Throwable e) { String message = String.format("Unable to start transaction-aware '%s' for transaction %d. ", - txAware.getTransactionAwareName(), currentTx.getWritePointer()); + txAware.getTransactionAwareName(), currentTx.getTransactionId()); LOG.warn(message, e); txClient.abort(currentTx); throw new TransactionFailureException(message, e); @@ -134,12 +134,12 @@ public void checkpoint() throws TransactionFailureException { txAware.startTx(currentTx); } } catch (TransactionNotInProgressException e) { - String message = String.format("Transaction %d is not in progress.", currentTx.getWritePointer()); + String message = String.format("Transaction %d is not in progress.", currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception } catch (Throwable e) { - String message = String.format("Exception from checkpoint for transaction %d.", currentTx.getWritePointer()); + String message = String.format("Exception from checkpoint for transaction %d.", currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception @@ -179,7 +179,7 @@ public void abort(TransactionFailureException cause) throws TransactionFailureEx } } catch (Throwable e) { String message = String.format("Unable to roll back changes in transaction-aware '%s' for transaction %d. ", - txAware.getTransactionAwareName(), currentTx.getWritePointer()); + txAware.getTransactionAwareName(), currentTx.getTransactionId()); LOG.warn(message, e); if (cause == null) { cause = new TransactionFailureException(message, e); @@ -190,7 +190,7 @@ public void abort(TransactionFailureException cause) throws TransactionFailureEx if (success) { txClient.abort(currentTx); } else { - txClient.invalidate(currentTx.getWritePointer()); + txClient.invalidate(currentTx.getTransactionId()); } if (cause != null) { throw cause; @@ -207,7 +207,7 @@ private void checkForConflicts() throws TransactionFailureException { changes.addAll(txAware.getTxChanges()); } catch (Throwable e) { String message = String.format("Unable to retrieve changes from transaction-aware '%s' for transaction %d. ", - txAware.getTransactionAwareName(), currentTx.getWritePointer()); + txAware.getTransactionAwareName(), currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception @@ -218,18 +218,18 @@ private void checkForConflicts() throws TransactionFailureException { try { canCommit = txClient.canCommit(currentTx, changes); } catch (TransactionNotInProgressException e) { - String message = String.format("Transaction %d is not in progress.", currentTx.getWritePointer()); + String message = String.format("Transaction %d is not in progress.", currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception } catch (Throwable e) { - String message = String.format("Exception from canCommit for transaction %d.", currentTx.getWritePointer()); + String message = String.format("Exception from canCommit for transaction %d.", currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception } if (!canCommit) { - String message = String.format("Conflict detected for transaction %d.", currentTx.getWritePointer()); + String message = String.format("Conflict detected for transaction %d.", currentTx.getTransactionId()); abort(new TransactionConflictException(message)); // abort will throw } @@ -247,7 +247,7 @@ private void persist() throws TransactionFailureException { } if (!success) { String message = String.format("Unable to persist changes of transaction-aware '%s' for transaction %d. ", - txAware.getTransactionAwareName(), currentTx.getWritePointer()); + txAware.getTransactionAwareName(), currentTx.getTransactionId()); if (cause == null) { LOG.warn(message); } else { @@ -264,18 +264,18 @@ private void commit() throws TransactionFailureException { try { commitSuccess = txClient.commit(currentTx); } catch (TransactionNotInProgressException e) { - String message = String.format("Transaction %d is not in progress.", currentTx.getWritePointer()); + String message = String.format("Transaction %d is not in progress.", currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception } catch (Throwable e) { - String message = String.format("Exception from commit for transaction %d.", currentTx.getWritePointer()); + String message = String.format("Exception from commit for transaction %d.", currentTx.getTransactionId()); LOG.warn(message, e); abort(new TransactionFailureException(message, e)); // abort will throw that exception } if (!commitSuccess) { - String message = String.format("Conflict detected for transaction %d.", currentTx.getWritePointer()); + String message = String.format("Conflict detected for transaction %d.", currentTx.getTransactionId()); abort(new TransactionConflictException(message)); // abort will throw } @@ -288,7 +288,7 @@ private void postCommit() throws TransactionFailureException { txAware.postTxCommit(); } catch (Throwable e) { String message = String.format("Unable to perform post-commit in transaction-aware '%s' for transaction %d. ", - txAware.getTransactionAwareName(), currentTx.getWritePointer()); + txAware.getTransactionAwareName(), currentTx.getTransactionId()); LOG.warn(message, e); cause = new TransactionFailureException(message, e); } diff --git a/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java b/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java index 54580fdb..ecc51373 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java +++ b/tephra-core/src/main/java/co/cask/tephra/TransactionManager.java @@ -756,11 +756,11 @@ private Transaction startTx(long expiration, TransactionType type) { ensureAvailable(); txid = getNextWritePointer(); tx = createTransaction(txid, type); - addInProgressAndAdvance(tx.getWritePointer(), tx.getVisibilityUpperBound(), expiration, type); + addInProgressAndAdvance(tx.getTransactionId(), tx.getVisibilityUpperBound(), expiration, type); } // appending to WAL out of global lock for concurrent performance // we should still be able to arrive at the same state even if log entries are out of order - appendToLog(TransactionEdit.createStarted(tx.getWritePointer(), tx.getVisibilityUpperBound(), expiration, type)); + appendToLog(TransactionEdit.createStarted(tx.getTransactionId(), tx.getVisibilityUpperBound(), expiration, type)); } finally { this.logReadLock.unlock(); } @@ -783,15 +783,15 @@ private void advanceWritePointer(long writePointer) { public boolean canCommit(Transaction tx, Collection changeIds) throws TransactionNotInProgressException { txMetricsCollector.rate("canCommit"); Stopwatch timer = new Stopwatch().start(); - if (inProgress.get(tx.getWritePointer()) == null) { + if (inProgress.get(tx.getTransactionId()) == null) { // invalid transaction, either this has timed out and moved to invalid, or something else is wrong. - if (invalid.contains(tx.getWritePointer())) { + if (invalid.contains(tx.getTransactionId())) { throw new TransactionNotInProgressException( String.format("canCommit() is called for transaction %d that is not in progress (it is known to be invalid)", - tx.getWritePointer())); + tx.getTransactionId())); } else { throw new TransactionNotInProgressException( - String.format("canCommit() is called for transaction %d that is not in progress", tx.getWritePointer())); + String.format("canCommit() is called for transaction %d that is not in progress", tx.getTransactionId())); } } @@ -808,9 +808,9 @@ public boolean canCommit(Transaction tx, Collection changeIds) throws Tr try { synchronized (this) { ensureAvailable(); - addCommittingChangeSet(tx.getWritePointer(), set); + addCommittingChangeSet(tx.getTransactionId(), set); } - appendToLog(TransactionEdit.createCommitting(tx.getWritePointer(), set)); + appendToLog(TransactionEdit.createCommitting(tx.getTransactionId(), set)); } finally { this.logReadLock.unlock(); } @@ -836,22 +836,22 @@ public boolean commit(Transaction tx) throws TransactionNotInProgressException { // we record commits at the first not-yet assigned transaction id to simplify clearing out change sets that // are no longer visible by any in-progress transactions commitPointer = lastWritePointer + 1; - if (inProgress.get(tx.getWritePointer()) == null) { + if (inProgress.get(tx.getTransactionId()) == null) { // invalid transaction, either this has timed out and moved to invalid, or something else is wrong. - if (invalid.contains(tx.getWritePointer())) { + if (invalid.contains(tx.getTransactionId())) { throw new TransactionNotInProgressException( String.format("canCommit() is called for transaction %d that is not in progress " + - "(it is known to be invalid)", tx.getWritePointer())); + "(it is known to be invalid)", tx.getTransactionId())); } else { throw new TransactionNotInProgressException( - String.format("canCommit() is called for transaction %d that is not in progress", tx.getWritePointer())); + String.format("canCommit() is called for transaction %d that is not in progress", tx.getTransactionId())); } } // these should be atomic // NOTE: whether we succeed or not we don't need to keep changes in committing state: same tx cannot // be attempted to commit twice - changeSet = committingChangeSets.remove(tx.getWritePointer()); + changeSet = committingChangeSets.remove(tx.getTransactionId()); if (changeSet != null) { // double-checking if there are conflicts: someone may have committed since canCommit check @@ -862,9 +862,9 @@ public boolean commit(Transaction tx) throws TransactionNotInProgressException { // no changes addToCommitted = false; } - doCommit(tx.getWritePointer(), changeSet, commitPointer, addToCommitted); + doCommit(tx.getTransactionId(), changeSet, commitPointer, addToCommitted); } - appendToLog(TransactionEdit.createCommitted(tx.getWritePointer(), changeSet, commitPointer, addToCommitted)); + appendToLog(TransactionEdit.createCommitted(tx.getTransactionId(), changeSet, commitPointer, addToCommitted)); } finally { this.logReadLock.unlock(); } @@ -916,9 +916,9 @@ public void abort(Transaction tx) { try { synchronized (this) { ensureAvailable(); - doAbort(tx.getWritePointer(), tx.getCheckpointWritePointers(), tx.getType()); + doAbort(tx.getTransactionId(), tx.getCheckpointWritePointers(), tx.getType()); } - appendToLog(TransactionEdit.createAborted(tx.getWritePointer(), tx.getType(), tx.getCheckpointWritePointers())); + appendToLog(TransactionEdit.createAborted(tx.getTransactionId(), tx.getType(), tx.getCheckpointWritePointers())); txMetricsCollector.histogram("abort.latency", (int) timer.elapsedMillis()); } finally { this.logReadLock.unlock(); @@ -1093,7 +1093,7 @@ public Transaction checkpoint(Transaction originalTx) throws TransactionNotInPro Stopwatch timer = new Stopwatch().start(); Transaction checkpointedTx = null; - long txId = originalTx.getWritePointer(); + long txId = originalTx.getTransactionId(); long newWritePointer = 0; // guard against changes to the transaction log while processing this.logReadLock.lock(); @@ -1159,7 +1159,7 @@ private boolean hasConflicts(Transaction tx, Set changeIds) { for (Map.Entry> changeSet : committedChangeSets.entrySet()) { // If commit time is greater than tx read-pointer, // basically not visible but committed means "tx committed after given tx was started" - if (changeSet.getKey() > tx.getWritePointer()) { + if (changeSet.getKey() > tx.getTransactionId()) { if (overlap(changeSet.getValue(), changeIds)) { return true; } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java index dd83da4d..f71218dc 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java @@ -28,10 +28,10 @@ public final class TransactionConverterUtils { public static TTransaction wrap(Transaction tx) { - return new TTransaction(tx.getWritePointer(), tx.getReadPointer(), + return new TTransaction(tx.getTransactionId(), tx.getReadPointer(), Longs.asList(tx.getInvalids()), Longs.asList(tx.getInProgress()), tx.getFirstShortInProgress(), getTTransactionType(tx.getType()), - tx.getCurrentWritePointer(), Longs.asList(tx.getCheckpointWritePointers())); + tx.getWritePointer(), Longs.asList(tx.getCheckpointWritePointers())); } public static Transaction unwrap(TTransaction thriftTx) { diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java index a3387d2d..6f5c81c0 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionServiceClient.java @@ -99,7 +99,7 @@ public static void doMain(boolean verbose, Configuration conf) throws Exception if (verbose) { LOG.info("Started tx details: " + tx.toString()); } else { - LOG.info("Started tx: " + tx.getWritePointer() + + LOG.info("Started tx: " + tx.getTransactionId() + ", readPointer: " + tx.getReadPointer() + ", invalids: " + tx.getInvalids().length + ", inProgress: " + tx.getInProgress().length); diff --git a/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java b/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java index 3fed8895..d9ff81e1 100644 --- a/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java +++ b/tephra-core/src/main/java/co/cask/tephra/util/TxUtils.java @@ -56,8 +56,8 @@ public static long getOldestVisibleTimestamp(Map ttlByFamily, Tran public static long getMaxVisibleTimestamp(Transaction tx) { // NOTE: +1 here because we want read up to writepointer inclusive, but timerange's end is exclusive // however, we also need to guard against overflow in the case write pointer is set to MAX_VALUE - return tx.getCurrentWritePointer() < Long.MAX_VALUE ? - tx.getCurrentWritePointer() + 1 : tx.getCurrentWritePointer(); + return tx.getWritePointer() < Long.MAX_VALUE ? + tx.getWritePointer() + 1 : tx.getWritePointer(); } /** diff --git a/tephra-core/src/test/java/co/cask/tephra/TransactionAdminTest.java b/tephra-core/src/test/java/co/cask/tephra/TransactionAdminTest.java index 506c7ec5..11636a91 100644 --- a/tephra-core/src/test/java/co/cask/tephra/TransactionAdminTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/TransactionAdminTest.java @@ -125,12 +125,12 @@ public void testPrintUsage() throws Exception { public void testTruncateInvalidTx() throws Exception { Transaction tx1 = txClient.startLong(); Transaction tx2 = txClient.startShort(); - txClient.invalidate(tx1.getWritePointer()); - txClient.invalidate(tx2.getWritePointer()); + txClient.invalidate(tx1.getTransactionId()); + txClient.invalidate(tx2.getTransactionId()); Assert.assertEquals(2, txClient.getInvalidSize()); TransactionAdmin txAdmin = new TransactionAdmin(new PrintStream(System.out), new PrintStream(System.err)); - int status = txAdmin.doMain(new String[]{"--truncate-invalid-tx", String.valueOf(tx2.getWritePointer())}, conf); + int status = txAdmin.doMain(new String[]{"--truncate-invalid-tx", String.valueOf(tx2.getTransactionId())}, conf); Assert.assertEquals(0, status); Assert.assertEquals(1, txClient.getInvalidSize()); } @@ -151,8 +151,8 @@ public void testTruncateInvalidTxBefore() throws Exception { // Assert no change to invalid size Assert.assertEquals(0, txClient.getInvalidSize()); - txClient.invalidate(tx1.getWritePointer()); - txClient.invalidate(tx2.getWritePointer()); + txClient.invalidate(tx1.getTransactionId()); + txClient.invalidate(tx2.getTransactionId()); Assert.assertEquals(2, txClient.getInvalidSize()); status = txAdmin.doMain(new String[]{"--truncate-invalid-tx-before", String.valueOf(beforeTx2)}, conf); @@ -164,7 +164,7 @@ public void testTruncateInvalidTxBefore() throws Exception { public void testGetInvalidTxSize() throws Exception { Transaction tx1 = txClient.startShort(); txClient.startLong(); - txClient.invalidate(tx1.getWritePointer()); + txClient.invalidate(tx1.getTransactionId()); ByteArrayOutputStream out = new ByteArrayOutputStream(); ByteArrayOutputStream err = new ByteArrayOutputStream(); diff --git a/tephra-core/src/test/java/co/cask/tephra/TransactionManagerTest.java b/tephra-core/src/test/java/co/cask/tephra/TransactionManagerTest.java index 07097fa8..2782d543 100644 --- a/tephra-core/src/test/java/co/cask/tephra/TransactionManagerTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/TransactionManagerTest.java @@ -98,9 +98,9 @@ public void testTransactionCleanup() throws Exception { // run another transaction Transaction txx = txm.startShort(); // verify the exclude - Assert.assertFalse(txx.isVisible(tx1.getWritePointer())); - Assert.assertFalse(txx.isVisible(tx2.getWritePointer())); - Assert.assertFalse(txx.isVisible(tx3.getWritePointer())); + Assert.assertFalse(txx.isVisible(tx1.getTransactionId())); + Assert.assertFalse(txx.isVisible(tx2.getTransactionId())); + Assert.assertFalse(txx.isVisible(tx3.getTransactionId())); // try to commit the last transaction that was started Assert.assertTrue(txm.canCommit(txx, Collections.singleton(new byte[] { 0x0a }))); Assert.assertTrue(txm.commit(txx)); @@ -133,7 +133,7 @@ public void testTransactionCleanup() throws Exception { // Only tx3 is invalid list as it was aborted and is long-running. tx1 is short one and it rolled back its changes // so it should NOT be in invalid list Assert.assertEquals(1, txm.getInvalidSize()); - Assert.assertEquals(tx3.getWritePointer(), (long) txm.getCurrentState().getInvalid().iterator().next()); + Assert.assertEquals(tx3.getTransactionId(), (long) txm.getCurrentState().getInvalid().iterator().next()); Assert.assertEquals(1, txm.getExcludedListSize()); } finally { txm.stopAndWait(); @@ -209,33 +209,34 @@ public void testTruncateInvalid() throws Exception { tx6 = txm1.startShort(); // invalidate tx1, tx2, tx5 and tx6 - txm1.invalidate(tx1.getWritePointer()); - txm1.invalidate(tx2.getWritePointer()); - txm1.invalidate(tx5.getWritePointer()); - txm1.invalidate(tx6.getWritePointer()); + txm1.invalidate(tx1.getTransactionId()); + txm1.invalidate(tx2.getTransactionId()); + txm1.invalidate(tx5.getTransactionId()); + txm1.invalidate(tx6.getTransactionId()); // tx1, tx2, tx5 and tx6 should be in invalid list Assert.assertEquals( - ImmutableList.of(tx1.getWritePointer(), tx2.getWritePointer(), tx5.getWritePointer(), tx6.getWritePointer()), + ImmutableList.of(tx1.getTransactionId(), tx2.getTransactionId(), tx5.getTransactionId(), + tx6.getTransactionId()), txm1.getCurrentState().getInvalid() ); // remove tx1 and tx6 from invalid list - Assert.assertTrue(txm1.truncateInvalidTx(ImmutableSet.of(tx1.getWritePointer(), tx6.getWritePointer()))); + Assert.assertTrue(txm1.truncateInvalidTx(ImmutableSet.of(tx1.getTransactionId(), tx6.getTransactionId()))); // only tx2 and tx5 should be in invalid list now - Assert.assertEquals(ImmutableList.of(tx2.getWritePointer(), tx5.getWritePointer()), + Assert.assertEquals(ImmutableList.of(tx2.getTransactionId(), tx5.getTransactionId()), txm1.getCurrentState().getInvalid()); // removing in-progress transactions should not have any effect - Assert.assertEquals(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()), + Assert.assertEquals(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()), txm1.getCurrentState().getInProgress().keySet()); - Assert.assertFalse(txm1.truncateInvalidTx(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()))); + Assert.assertFalse(txm1.truncateInvalidTx(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()))); // no change to in-progress - Assert.assertEquals(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()), + Assert.assertEquals(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()), txm1.getCurrentState().getInProgress().keySet()); // no change to invalid list - Assert.assertEquals(ImmutableList.of(tx2.getWritePointer(), tx5.getWritePointer()), + Assert.assertEquals(ImmutableList.of(tx2.getTransactionId(), tx5.getTransactionId()), txm1.getCurrentState().getInvalid()); // Test transaction edit logs replay @@ -243,9 +244,9 @@ public void testTruncateInvalid() throws Exception { // and all logs can be replayed. txm2 = new TransactionManager(testConf, storage, new TxMetricsCollector()); txm2.startAndWait(); - Assert.assertEquals(ImmutableList.of(tx2.getWritePointer(), tx5.getWritePointer()), + Assert.assertEquals(ImmutableList.of(tx2.getTransactionId(), tx5.getTransactionId()), txm2.getCurrentState().getInvalid()); - Assert.assertEquals(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()), + Assert.assertEquals(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()), txm2.getCurrentState().getInProgress().keySet()); } finally { txm1.stopAndWait(); @@ -289,14 +290,15 @@ public void testTruncateInvalidBeforeTime() throws Exception { tx6 = txm1.startShort(); // invalidate tx1, tx2, tx5 and tx6 - txm1.invalidate(tx1.getWritePointer()); - txm1.invalidate(tx2.getWritePointer()); - txm1.invalidate(tx5.getWritePointer()); - txm1.invalidate(tx6.getWritePointer()); + txm1.invalidate(tx1.getTransactionId()); + txm1.invalidate(tx2.getTransactionId()); + txm1.invalidate(tx5.getTransactionId()); + txm1.invalidate(tx6.getTransactionId()); // tx1, tx2, tx5 and tx6 should be in invalid list Assert.assertEquals( - ImmutableList.of(tx1.getWritePointer(), tx2.getWritePointer(), tx5.getWritePointer(), tx6.getWritePointer()), + ImmutableList.of(tx1.getTransactionId(), tx2.getTransactionId(), tx5.getTransactionId(), + tx6.getTransactionId()), txm1.getCurrentState().getInvalid() ); @@ -304,11 +306,11 @@ public void testTruncateInvalidBeforeTime() throws Exception { Assert.assertTrue(txm1.truncateInvalidTxBefore(timeBeforeTx3)); // only tx5 and tx6 should be in invalid list now - Assert.assertEquals(ImmutableList.of(tx5.getWritePointer(), tx6.getWritePointer()), + Assert.assertEquals(ImmutableList.of(tx5.getTransactionId(), tx6.getTransactionId()), txm1.getCurrentState().getInvalid()); // removing invalid transactions before tx5 should throw exception since tx3 and tx4 are in-progress - Assert.assertEquals(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()), + Assert.assertEquals(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()), txm1.getCurrentState().getInProgress().keySet()); try { txm1.truncateInvalidTxBefore(timeBeforeTx5); @@ -317,10 +319,10 @@ public void testTruncateInvalidBeforeTime() throws Exception { // Expected exception } // no change to in-progress - Assert.assertEquals(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()), + Assert.assertEquals(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()), txm1.getCurrentState().getInProgress().keySet()); // no change to invalid list - Assert.assertEquals(ImmutableList.of(tx5.getWritePointer(), tx6.getWritePointer()), + Assert.assertEquals(ImmutableList.of(tx5.getTransactionId(), tx6.getTransactionId()), txm1.getCurrentState().getInvalid()); // Test transaction edit logs replay @@ -328,9 +330,9 @@ public void testTruncateInvalidBeforeTime() throws Exception { // and all logs can be replayed. txm2 = new TransactionManager(testConf, storage, new TxMetricsCollector()); txm2.startAndWait(); - Assert.assertEquals(ImmutableList.of(tx5.getWritePointer(), tx6.getWritePointer()), + Assert.assertEquals(ImmutableList.of(tx5.getTransactionId(), tx6.getTransactionId()), txm2.getCurrentState().getInvalid()); - Assert.assertEquals(ImmutableSet.of(tx3.getWritePointer(), tx4.getWritePointer()), + Assert.assertEquals(ImmutableSet.of(tx3.getTransactionId(), tx4.getTransactionId()), txm2.getCurrentState().getInProgress().keySet()); } finally { txm1.stopAndWait(); diff --git a/tephra-core/src/test/java/co/cask/tephra/TransactionSystemTest.java b/tephra-core/src/test/java/co/cask/tephra/TransactionSystemTest.java index ba13a30a..1bb938e5 100644 --- a/tephra-core/src/test/java/co/cask/tephra/TransactionSystemTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/TransactionSystemTest.java @@ -146,7 +146,7 @@ public void testUseNotStarted() throws Exception { Assert.assertTrue(client.commit(tx1)); // we know this is one is older than current writePointer and was not used - Transaction txOld = new Transaction(tx1.getReadPointer(), tx1.getWritePointer() - 1, + Transaction txOld = new Transaction(tx1.getReadPointer(), tx1.getTransactionId() - 1, new long[] {}, new long[] {}, Transaction.NO_TX_IN_PROGRESS, TransactionType.SHORT); try { @@ -165,7 +165,7 @@ public void testUseNotStarted() throws Exception { client.abort(txOld); // we know this is one is newer than current readPointer and was not used - Transaction txNew = new Transaction(tx1.getReadPointer(), tx1.getWritePointer() + 1, + Transaction txNew = new Transaction(tx1.getReadPointer(), tx1.getTransactionId() + 1, new long[] {}, new long[] {}, Transaction.NO_TX_IN_PROGRESS, TransactionType.SHORT); try { @@ -202,12 +202,12 @@ public void testInvalidateTx() throws Exception { // Invalidate an in-progress tx Transaction tx1 = client.startShort(); client.canCommit(tx1, asList(C1, C2)); - Assert.assertTrue(client.invalidate(tx1.getWritePointer())); + Assert.assertTrue(client.invalidate(tx1.getTransactionId())); // Cannot invalidate a committed tx Transaction tx2 = client.startShort(); client.canCommit(tx2, asList(C3, C4)); client.commit(tx2); - Assert.assertFalse(client.invalidate(tx2.getWritePointer())); + Assert.assertFalse(client.invalidate(tx2.getTransactionId())); } @Test @@ -237,7 +237,7 @@ public void testResetState() throws Exception { // confirm that transaction IDs are not reset Transaction txPostReset = client.startShort(); Assert.assertTrue("New tx ID should be greater than last ID before reset", - txPostReset.getWritePointer() > txPreReset.getWritePointer()); + txPostReset.getTransactionId() > txPreReset.getTransactionId()); } @Test @@ -248,16 +248,16 @@ public void testTruncateInvalidTx() throws Exception { Transaction tx2 = client.startShort(); Transaction tx3 = client.startLong(); - client.invalidate(tx1.getWritePointer()); - client.invalidate(tx2.getWritePointer()); - client.invalidate(tx3.getWritePointer()); + client.invalidate(tx1.getTransactionId()); + client.invalidate(tx2.getTransactionId()); + client.invalidate(tx3.getTransactionId()); // Remove tx2 and tx3 from invalid list - Assert.assertTrue(client.truncateInvalidTx(ImmutableSet.of(tx2.getWritePointer(), tx3.getWritePointer()))); + Assert.assertTrue(client.truncateInvalidTx(ImmutableSet.of(tx2.getTransactionId(), tx3.getTransactionId()))); Transaction tx = client.startShort(); // Only tx1 should be in invalid list now - Assert.assertArrayEquals(new long[] {tx1.getWritePointer()}, tx.getInvalids()); + Assert.assertArrayEquals(new long[] {tx1.getTransactionId()}, tx.getInvalids()); client.abort(tx); } @@ -282,16 +282,16 @@ public void testTruncateInvalidTxBefore() throws Exception { } // Invalidate all of them - client.invalidate(tx1.getWritePointer()); - client.invalidate(tx2.getWritePointer()); - client.invalidate(tx3.getWritePointer()); + client.invalidate(tx1.getTransactionId()); + client.invalidate(tx2.getTransactionId()); + client.invalidate(tx3.getTransactionId()); // Remove transactions before time beforeTx3 Assert.assertTrue(client.truncateInvalidTxBefore(beforeTx3)); Transaction tx = client.startShort(); // Only tx3 should be in invalid list now - Assert.assertArrayEquals(new long[] {tx3.getWritePointer()}, tx.getInvalids()); + Assert.assertArrayEquals(new long[] {tx3.getTransactionId()}, tx.getInvalids()); client.abort(tx); } @@ -305,9 +305,9 @@ public void testGetInvalidSize() throws Exception { Assert.assertEquals(0, client.getInvalidSize()); - client.invalidate(tx1.getWritePointer()); - client.invalidate(tx2.getWritePointer()); - client.invalidate(tx3.getWritePointer()); + client.invalidate(tx1.getTransactionId()); + client.invalidate(tx2.getTransactionId()); + client.invalidate(tx3.getTransactionId()); Assert.assertEquals(3, client.getInvalidSize()); } diff --git a/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java b/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java index 8ae04edf..e5c499b4 100644 --- a/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/persist/AbstractTransactionStateStorageTest.java @@ -163,19 +163,19 @@ public void testTransactionManagerPersistence() throws Exception { Assert.assertTrue(txManager.commit(tx2)); // start another transaction, must be greater than tx3 Transaction tx4 = txManager.startShort(); - Assert.assertTrue(tx4.getWritePointer() > tx3.getWritePointer()); + Assert.assertTrue(tx4.getTransactionId() > tx3.getTransactionId()); // tx1 must be visble from tx2, but tx3 and tx4 must not - Assert.assertTrue(tx2.isVisible(tx1.getWritePointer())); - Assert.assertFalse(tx2.isVisible(tx3.getWritePointer())); - Assert.assertFalse(tx2.isVisible(tx4.getWritePointer())); + Assert.assertTrue(tx2.isVisible(tx1.getTransactionId())); + Assert.assertFalse(tx2.isVisible(tx3.getTransactionId())); + Assert.assertFalse(tx2.isVisible(tx4.getTransactionId())); // add same change for tx3 Assert.assertFalse(txManager.canCommit(tx3, Collections.singleton(b))); // check visibility with new xaction Transaction tx5 = txManager.startShort(); - Assert.assertTrue(tx5.isVisible(tx1.getWritePointer())); - Assert.assertTrue(tx5.isVisible(tx2.getWritePointer())); - Assert.assertFalse(tx5.isVisible(tx3.getWritePointer())); - Assert.assertFalse(tx5.isVisible(tx4.getWritePointer())); + Assert.assertTrue(tx5.isVisible(tx1.getTransactionId())); + Assert.assertTrue(tx5.isVisible(tx2.getTransactionId())); + Assert.assertFalse(tx5.isVisible(tx3.getTransactionId())); + Assert.assertFalse(tx5.isVisible(tx4.getTransactionId())); // can commit tx3? txManager.abort(tx3); txManager.abort(tx4); @@ -204,7 +204,7 @@ public void testTransactionManagerPersistence() throws Exception { // get a new transaction and verify it is greater Transaction txAfter = txManager.startShort(); - Assert.assertTrue(txAfter.getWritePointer() > tx.getWritePointer()); + Assert.assertTrue(txAfter.getTransactionId() > tx.getTransactionId()); } finally { if (storage != null) { storage.stopAndWait(); diff --git a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java index a123c3f3..84bfac14 100644 --- a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java +++ b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/TransactionAwareHTable.java @@ -499,14 +499,14 @@ private Scan transactionalizeAction(Scan scan) throws IOException { } private Put transactionalizeAction(Put put) throws IOException { - Put txPut = new Put(put.getRow(), tx.getCurrentWritePointer()); + Put txPut = new Put(put.getRow(), tx.getWritePointer()); Set>> familyMap = put.getFamilyCellMap().entrySet(); if (!familyMap.isEmpty()) { for (Map.Entry> family : familyMap) { List familyValues = family.getValue(); if (!familyValues.isEmpty()) { for (Cell value : familyValues) { - txPut.add(value.getFamily(), value.getQualifier(), tx.getCurrentWritePointer(), value.getValue()); + txPut.add(value.getFamily(), value.getQualifier(), tx.getWritePointer(), value.getValue()); addToChangeSet(txPut.getRow(), value.getFamily(), value.getQualifier()); } } @@ -521,7 +521,7 @@ private Put transactionalizeAction(Put put) throws IOException { } private Delete transactionalizeAction(Delete delete) throws IOException { - long transactionTimestamp = tx.getCurrentWritePointer(); + long transactionTimestamp = tx.getWritePointer(); byte[] deleteRow = delete.getRow(); Delete txDelete = new Delete(deleteRow, transactionTimestamp); diff --git a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java index 228c7053..066d5185 100644 --- a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java +++ b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -768,22 +767,22 @@ public void testCheckpoint() throws Exception { transactionContext.checkpoint(); Transaction postCheckpointTx = transactionContext.getCurrentTransaction(); - assertEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer()); - assertNotEquals(origTx.getCurrentWritePointer(), postCheckpointTx.getCurrentWritePointer()); + assertEquals(origTx.getTransactionId(), postCheckpointTx.getTransactionId()); + assertNotEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer()); long[] checkpointPtrs = postCheckpointTx.getCheckpointWritePointers(); assertEquals(1, checkpointPtrs.length); - assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs[0]); + assertEquals(postCheckpointTx.getWritePointer(), checkpointPtrs[0]); transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); transactionContext.checkpoint(); Transaction postCheckpointTx2 = transactionContext.getCurrentTransaction(); - assertEquals(origTx.getWritePointer(), postCheckpointTx2.getWritePointer()); - assertNotEquals(postCheckpointTx.getCurrentWritePointer(), postCheckpointTx2.getCurrentWritePointer()); + assertEquals(origTx.getTransactionId(), postCheckpointTx2.getTransactionId()); + assertNotEquals(postCheckpointTx.getWritePointer(), postCheckpointTx2.getWritePointer()); long[] checkpointPtrs2 = postCheckpointTx2.getCheckpointWritePointers(); assertEquals(2, checkpointPtrs2.length); - assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs2[0]); - assertEquals(postCheckpointTx2.getCurrentWritePointer(), checkpointPtrs2[1]); + assertEquals(postCheckpointTx.getWritePointer(), checkpointPtrs2[0]); + assertEquals(postCheckpointTx2.getWritePointer(), checkpointPtrs2[1]); transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); @@ -883,7 +882,7 @@ public void testCheckpointInvalidate() throws Exception { transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); TransactionSystemClient txClient = new InMemoryTxSystemClient(txManager); - txClient.invalidate(transactionContext.getCurrentTransaction().getWritePointer()); + txClient.invalidate(transactionContext.getCurrentTransaction().getTransactionId()); // check that writes are not visible TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); @@ -892,9 +891,9 @@ public void testCheckpointInvalidate() throws Exception { Transaction newTx = txContext2.getCurrentTransaction(); // all 3 writes pointers from the previous transaction should now be excluded - assertTrue(newTx.isExcluded(origTx.getCurrentWritePointer())); - assertTrue(newTx.isExcluded(checkpointTx1.getCurrentWritePointer())); - assertTrue(newTx.isExcluded(checkpointTx2.getCurrentWritePointer())); + assertTrue(newTx.isExcluded(origTx.getWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx1.getWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx2.getWritePointer())); verifyRow(txTable2, TestBytes.row, null); verifyRow(txTable2, TestBytes.row2, null); diff --git a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/coprocessor/TransactionVisibilityFilterTest.java b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/coprocessor/TransactionVisibilityFilterTest.java index 09de6239..a50e9139 100644 --- a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/coprocessor/TransactionVisibilityFilterTest.java +++ b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/coprocessor/TransactionVisibilityFilterTest.java @@ -65,7 +65,7 @@ public void testFiltering() throws Exception { Transaction tx3 = txManager.startShort(); Transaction tx4 = txManager.startShort(); - txManager.invalidate(tx4.getWritePointer()); + txManager.invalidate(tx4.getTransactionId()); Transaction tx5 = txManager.startShort(); assertTrue(txManager.canCommit(tx5, EMPTY_CHANGESET)); @@ -77,15 +77,15 @@ public void testFiltering() throws Exception { Filter filter = createFilter(tx6, ttls); assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, - filter.filterKeyValue(newKeyValue("row1", "val1", tx6.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx6.getTransactionId()))); assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, - filter.filterKeyValue(newKeyValue("row1", "val1", tx5.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx5.getTransactionId()))); assertEquals(Filter.ReturnCode.SKIP, - filter.filterKeyValue(newKeyValue("row1", "val1", tx4.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx4.getTransactionId()))); assertEquals(Filter.ReturnCode.SKIP, - filter.filterKeyValue(newKeyValue("row1", "val1", tx3.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx3.getTransactionId()))); assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, - filter.filterKeyValue(newKeyValue("row1", "val1", tx2.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx2.getTransactionId()))); } /** diff --git a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java index e2d480d8..48637284 100644 --- a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java +++ b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/TransactionAwareHTable.java @@ -528,14 +528,14 @@ private Scan transactionalizeAction(Scan scan) throws IOException { } private Put transactionalizeAction(Put put) throws IOException { - Put txPut = new Put(put.getRow(), tx.getCurrentWritePointer()); + Put txPut = new Put(put.getRow(), tx.getWritePointer()); Set>> familyMap = put.getFamilyCellMap().entrySet(); if (!familyMap.isEmpty()) { for (Map.Entry> family : familyMap) { List familyValues = family.getValue(); if (!familyValues.isEmpty()) { for (Cell value : familyValues) { - txPut.add(value.getFamily(), value.getQualifier(), tx.getCurrentWritePointer(), value.getValue()); + txPut.add(value.getFamily(), value.getQualifier(), tx.getWritePointer(), value.getValue()); addToChangeSet(txPut.getRow(), value.getFamily(), value.getQualifier()); } } @@ -550,7 +550,7 @@ private Put transactionalizeAction(Put put) throws IOException { } private Delete transactionalizeAction(Delete delete) throws IOException { - long transactionTimestamp = tx.getCurrentWritePointer(); + long transactionTimestamp = tx.getWritePointer(); byte[] deleteRow = delete.getRow(); Delete txDelete = new Delete(deleteRow, transactionTimestamp); diff --git a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java index 18decde0..db154764 100644 --- a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java +++ b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java @@ -768,22 +768,22 @@ public void testCheckpoint() throws Exception { transactionContext.checkpoint(); Transaction postCheckpointTx = transactionContext.getCurrentTransaction(); - assertEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer()); - assertNotEquals(origTx.getCurrentWritePointer(), postCheckpointTx.getCurrentWritePointer()); + assertEquals(origTx.getTransactionId(), postCheckpointTx.getTransactionId()); + assertNotEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer()); long[] checkpointPtrs = postCheckpointTx.getCheckpointWritePointers(); assertEquals(1, checkpointPtrs.length); - assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs[0]); + assertEquals(postCheckpointTx.getWritePointer(), checkpointPtrs[0]); transactionAwareHTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)); transactionContext.checkpoint(); Transaction postCheckpointTx2 = transactionContext.getCurrentTransaction(); - assertEquals(origTx.getWritePointer(), postCheckpointTx2.getWritePointer()); - assertNotEquals(postCheckpointTx.getCurrentWritePointer(), postCheckpointTx2.getCurrentWritePointer()); + assertEquals(origTx.getTransactionId(), postCheckpointTx2.getTransactionId()); + assertNotEquals(postCheckpointTx.getWritePointer(), postCheckpointTx2.getWritePointer()); long[] checkpointPtrs2 = postCheckpointTx2.getCheckpointWritePointers(); assertEquals(2, checkpointPtrs2.length); - assertEquals(postCheckpointTx.getCurrentWritePointer(), checkpointPtrs2[0]); - assertEquals(postCheckpointTx2.getCurrentWritePointer(), checkpointPtrs2[1]); + assertEquals(postCheckpointTx.getWritePointer(), checkpointPtrs2[0]); + assertEquals(postCheckpointTx2.getWritePointer(), checkpointPtrs2[1]); transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); @@ -883,7 +883,7 @@ public void testCheckpointInvalidate() throws Exception { transactionAwareHTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value)); TransactionSystemClient txClient = new InMemoryTxSystemClient(txManager); - txClient.invalidate(transactionContext.getCurrentTransaction().getWritePointer()); + txClient.invalidate(transactionContext.getCurrentTransaction().getTransactionId()); // check that writes are not visible TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); @@ -892,9 +892,9 @@ public void testCheckpointInvalidate() throws Exception { Transaction newTx = txContext2.getCurrentTransaction(); // all 3 writes pointers from the previous transaction should now be excluded - assertTrue(newTx.isExcluded(origTx.getCurrentWritePointer())); - assertTrue(newTx.isExcluded(checkpointTx1.getCurrentWritePointer())); - assertTrue(newTx.isExcluded(checkpointTx2.getCurrentWritePointer())); + assertTrue(newTx.isExcluded(origTx.getWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx1.getWritePointer())); + assertTrue(newTx.isExcluded(checkpointTx2.getWritePointer())); verifyRow(txTable2, TestBytes.row, null); verifyRow(txTable2, TestBytes.row2, null); diff --git a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/coprocessor/TransactionVisibilityFilterTest.java b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/coprocessor/TransactionVisibilityFilterTest.java index f00dfd15..94cbdcfd 100644 --- a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/coprocessor/TransactionVisibilityFilterTest.java +++ b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/coprocessor/TransactionVisibilityFilterTest.java @@ -65,7 +65,7 @@ public void testFiltering() throws Exception { Transaction tx3 = txManager.startShort(); Transaction tx4 = txManager.startShort(); - txManager.invalidate(tx4.getWritePointer()); + txManager.invalidate(tx4.getTransactionId()); Transaction tx5 = txManager.startShort(); assertTrue(txManager.canCommit(tx5, EMPTY_CHANGESET)); @@ -77,15 +77,15 @@ public void testFiltering() throws Exception { Filter filter = createFilter(tx6, ttls); assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, - filter.filterKeyValue(newKeyValue("row1", "val1", tx6.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx6.getTransactionId()))); assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, - filter.filterKeyValue(newKeyValue("row1", "val1", tx5.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx5.getTransactionId()))); assertEquals(Filter.ReturnCode.SKIP, - filter.filterKeyValue(newKeyValue("row1", "val1", tx4.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx4.getTransactionId()))); assertEquals(Filter.ReturnCode.SKIP, - filter.filterKeyValue(newKeyValue("row1", "val1", tx3.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx3.getTransactionId()))); assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, - filter.filterKeyValue(newKeyValue("row1", "val1", tx2.getWritePointer()))); + filter.filterKeyValue(newKeyValue("row1", "val1", tx2.getTransactionId()))); } /** From 1a50b3eb9f67771a22747303ca803346f3d47749 Mon Sep 17 00:00:00 2001 From: Gary Helmling Date: Wed, 20 May 2015 16:39:29 -0700 Subject: [PATCH 3/5] TEPHRA-96 Checkpoints should only update the current transaction, not reset state --- .../main/java/co/cask/tephra/TransactionAware.java | 13 ++++++++++++- .../main/java/co/cask/tephra/TransactionAwares.java | 7 +++++++ .../cask/tephra/AbstractTransactionAwareTable.java | 5 +++++ .../java/co/cask/tephra/TransactionContext.java | 2 +- .../java/co/cask/tephra/TransactionContextTest.java | 5 +++++ .../co/cask/tephra/TransactionExecutorTest.java | 5 +++++ 6 files changed, 35 insertions(+), 2 deletions(-) diff --git a/tephra-api/src/main/java/co/cask/tephra/TransactionAware.java b/tephra-api/src/main/java/co/cask/tephra/TransactionAware.java index df97d4da..de7c9468 100644 --- a/tephra-api/src/main/java/co/cask/tephra/TransactionAware.java +++ b/tephra-api/src/main/java/co/cask/tephra/TransactionAware.java @@ -54,12 +54,23 @@ // todo: add onCommitted() - so that e.g. hbase table can do *actual* deletes at this point public interface TransactionAware { /** - * Called when new transaction has started. + * Called when new transaction has started. This may reset any state which has been left behind by the previous + * transaction. + * * @param tx transaction info */ // todo: rename to onTxStart() void startTx(Transaction tx); + /** + * Called when the state of the current transaction has been updated. This should replace any reference to the + * current {@link Transaction} held by this {@code TransactionAware}, but should not reset + * any state (such as the write change sets) that is currently maintained. + * + * @param tx the updated transaction + */ + void updateTx(Transaction tx); + /** * @return changes made by current transaction to be used for conflicts detection before commit. */ diff --git a/tephra-api/src/main/java/co/cask/tephra/TransactionAwares.java b/tephra-api/src/main/java/co/cask/tephra/TransactionAwares.java index d7d682f3..ab1654c6 100644 --- a/tephra-api/src/main/java/co/cask/tephra/TransactionAwares.java +++ b/tephra-api/src/main/java/co/cask/tephra/TransactionAwares.java @@ -53,6 +53,13 @@ public void startTx(Transaction tx) { } } + @Override + public void updateTx(Transaction tx) { + for (TransactionAware txAware : this) { + txAware.updateTx(tx); + } + } + @Override public Collection getTxChanges() { List changes = new ArrayList(); diff --git a/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java b/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java index 38da0115..9d0b2472 100644 --- a/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java +++ b/tephra-core/src/main/java/co/cask/tephra/AbstractTransactionAwareTable.java @@ -71,6 +71,11 @@ public void startTx(Transaction tx) { this.tx = tx; } + @Override + public void updateTx(Transaction tx) { + this.tx = tx; + } + @Override public Collection getTxChanges() { if (conflictLevel == TxConstants.ConflictDetection.NONE) { diff --git a/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java b/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java index 2d96061d..4c77cf0b 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java +++ b/tephra-core/src/main/java/co/cask/tephra/TransactionContext.java @@ -131,7 +131,7 @@ public void checkpoint() throws TransactionFailureException { currentTx = txClient.checkpoint(currentTx); // update the current transaction with all TransactionAwares for (TransactionAware txAware : txAwares) { - txAware.startTx(currentTx); + txAware.updateTx(currentTx); } } catch (TransactionNotInProgressException e) { String message = String.format("Transaction %d is not in progress.", currentTx.getTransactionId()); diff --git a/tephra-core/src/test/java/co/cask/tephra/TransactionContextTest.java b/tephra-core/src/test/java/co/cask/tephra/TransactionContextTest.java index a560f671..307cbd68 100644 --- a/tephra-core/src/test/java/co/cask/tephra/TransactionContextTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/TransactionContextTest.java @@ -497,6 +497,11 @@ public void startTx(Transaction tx) { } } + @Override + public void updateTx(Transaction tx) { + this.tx = tx; + } + @Override public Collection getTxChanges() { checked = true; diff --git a/tephra-core/src/test/java/co/cask/tephra/TransactionExecutorTest.java b/tephra-core/src/test/java/co/cask/tephra/TransactionExecutorTest.java index 66d5dac6..416ef6e0 100644 --- a/tephra-core/src/test/java/co/cask/tephra/TransactionExecutorTest.java +++ b/tephra-core/src/test/java/co/cask/tephra/TransactionExecutorTest.java @@ -463,6 +463,11 @@ public void startTx(Transaction tx) { } } + @Override + public void updateTx(Transaction tx) { + this.tx = tx; + } + @Override public Collection getTxChanges() { checked = true; From 9dd456630c7df4d7eccdb5914eafcfab04765596 Mon Sep 17 00:00:00 2001 From: Gary Helmling Date: Wed, 20 May 2015 18:54:27 -0700 Subject: [PATCH 4/5] TEPHRA-98 Rename thrift fields for transactionId and writePointer --- .../TransactionConverterUtils.java | 2 +- .../distributed/thrift/TTransaction.java | 208 +++++++++--------- .../thrift/TTransactionServer.java | 12 +- .../src/main/thrift/transaction.thrift | 4 +- 4 files changed, 113 insertions(+), 113 deletions(-) diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java index f71218dc..ecc79f19 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java @@ -35,7 +35,7 @@ public static TTransaction wrap(Transaction tx) { } public static Transaction unwrap(TTransaction thriftTx) { - return new Transaction(thriftTx.getReadPointer(), thriftTx.getWritePointer(), thriftTx.getCurrentWritePointer(), + return new Transaction(thriftTx.getReadPointer(), thriftTx.getTransactionId(), thriftTx.getWritePointer(), Longs.toArray(thriftTx.getInvalids()), Longs.toArray(thriftTx.getInProgress()), thriftTx.getFirstShort(), getTransactionType(thriftTx.getType()), Longs.toArray(thriftTx.getCheckpointWritePointers())); diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java index 2d7cc79f..068e8e26 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java @@ -49,13 +49,13 @@ public class TTransaction implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTransaction"); - private static final org.apache.thrift.protocol.TField WRITE_POINTER_FIELD_DESC = new org.apache.thrift.protocol.TField("writePointer", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField TRANSACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("transactionId", org.apache.thrift.protocol.TType.I64, (short)1); private static final org.apache.thrift.protocol.TField READ_POINTER_FIELD_DESC = new org.apache.thrift.protocol.TField("readPointer", org.apache.thrift.protocol.TType.I64, (short)2); private static final org.apache.thrift.protocol.TField INVALIDS_FIELD_DESC = new org.apache.thrift.protocol.TField("invalids", org.apache.thrift.protocol.TType.LIST, (short)3); private static final org.apache.thrift.protocol.TField IN_PROGRESS_FIELD_DESC = new org.apache.thrift.protocol.TField("inProgress", org.apache.thrift.protocol.TType.LIST, (short)4); private static final org.apache.thrift.protocol.TField FIRST_SHORT_FIELD_DESC = new org.apache.thrift.protocol.TField("firstShort", org.apache.thrift.protocol.TType.I64, (short)5); private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)6); - private static final org.apache.thrift.protocol.TField CURRENT_WRITE_POINTER_FIELD_DESC = new org.apache.thrift.protocol.TField("currentWritePointer", org.apache.thrift.protocol.TType.I64, (short)7); + private static final org.apache.thrift.protocol.TField WRITE_POINTER_FIELD_DESC = new org.apache.thrift.protocol.TField("writePointer", org.apache.thrift.protocol.TType.I64, (short)7); private static final org.apache.thrift.protocol.TField CHECKPOINT_WRITE_POINTERS_FIELD_DESC = new org.apache.thrift.protocol.TField("checkpointWritePointers", org.apache.thrift.protocol.TType.LIST, (short)8); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); @@ -64,7 +64,7 @@ public class TTransaction implements org.apache.thrift.TBase invalids; // required public List inProgress; // required @@ -74,12 +74,12 @@ public class TTransaction implements org.apache.thrift.TBase checkpointWritePointers; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - WRITE_POINTER((short)1, "writePointer"), + TRANSACTION_ID((short)1, "transactionId"), READ_POINTER((short)2, "readPointer"), INVALIDS((short)3, "invalids"), IN_PROGRESS((short)4, "inProgress"), @@ -89,7 +89,7 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { * @see TTransactionType */ TYPE((short)6, "type"), - CURRENT_WRITE_POINTER((short)7, "currentWritePointer"), + WRITE_POINTER((short)7, "writePointer"), CHECKPOINT_WRITE_POINTERS((short)8, "checkpointWritePointers"); private static final Map byName = new HashMap(); @@ -105,8 +105,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // WRITE_POINTER - return WRITE_POINTER; + case 1: // TRANSACTION_ID + return TRANSACTION_ID; case 2: // READ_POINTER return READ_POINTER; case 3: // INVALIDS @@ -117,8 +117,8 @@ public static _Fields findByThriftId(int fieldId) { return FIRST_SHORT; case 6: // TYPE return TYPE; - case 7: // CURRENT_WRITE_POINTER - return CURRENT_WRITE_POINTER; + case 7: // WRITE_POINTER + return WRITE_POINTER; case 8: // CHECKPOINT_WRITE_POINTERS return CHECKPOINT_WRITE_POINTERS; default: @@ -161,15 +161,15 @@ public String getFieldName() { } // isset id assignments - private static final int __WRITEPOINTER_ISSET_ID = 0; + private static final int __TRANSACTIONID_ISSET_ID = 0; private static final int __READPOINTER_ISSET_ID = 1; private static final int __FIRSTSHORT_ISSET_ID = 2; - private static final int __CURRENTWRITEPOINTER_ISSET_ID = 3; + private static final int __WRITEPOINTER_ISSET_ID = 3; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.WRITE_POINTER, new org.apache.thrift.meta_data.FieldMetaData("writePointer", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.TRANSACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("transactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.READ_POINTER, new org.apache.thrift.meta_data.FieldMetaData("readPointer", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); @@ -183,7 +183,7 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTransactionType.class))); - tmpMap.put(_Fields.CURRENT_WRITE_POINTER, new org.apache.thrift.meta_data.FieldMetaData("currentWritePointer", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.WRITE_POINTER, new org.apache.thrift.meta_data.FieldMetaData("writePointer", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.CHECKPOINT_WRITE_POINTERS, new org.apache.thrift.meta_data.FieldMetaData("checkpointWritePointers", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, @@ -196,18 +196,18 @@ public TTransaction() { } public TTransaction( - long writePointer, + long transactionId, long readPointer, List invalids, List inProgress, long firstShort, TTransactionType type, - long currentWritePointer, + long writePointer, List checkpointWritePointers) { this(); - this.writePointer = writePointer; - setWritePointerIsSet(true); + this.transactionId = transactionId; + setTransactionIdIsSet(true); this.readPointer = readPointer; setReadPointerIsSet(true); this.invalids = invalids; @@ -215,8 +215,8 @@ public TTransaction( this.firstShort = firstShort; setFirstShortIsSet(true); this.type = type; - this.currentWritePointer = currentWritePointer; - setCurrentWritePointerIsSet(true); + this.writePointer = writePointer; + setWritePointerIsSet(true); this.checkpointWritePointers = checkpointWritePointers; } @@ -225,7 +225,7 @@ public TTransaction( */ public TTransaction(TTransaction other) { __isset_bitfield = other.__isset_bitfield; - this.writePointer = other.writePointer; + this.transactionId = other.transactionId; this.readPointer = other.readPointer; if (other.isSetInvalids()) { List __this__invalids = new ArrayList(); @@ -245,7 +245,7 @@ public TTransaction(TTransaction other) { if (other.isSetType()) { this.type = other.type; } - this.currentWritePointer = other.currentWritePointer; + this.writePointer = other.writePointer; if (other.isSetCheckpointWritePointers()) { List __this__checkpointWritePointers = new ArrayList(); for (Long other_element : other.checkpointWritePointers) { @@ -261,8 +261,8 @@ public TTransaction deepCopy() { @Override public void clear() { - setWritePointerIsSet(false); - this.writePointer = 0; + setTransactionIdIsSet(false); + this.transactionId = 0; setReadPointerIsSet(false); this.readPointer = 0; this.invalids = null; @@ -270,32 +270,32 @@ public void clear() { setFirstShortIsSet(false); this.firstShort = 0; this.type = null; - setCurrentWritePointerIsSet(false); - this.currentWritePointer = 0; + setWritePointerIsSet(false); + this.writePointer = 0; this.checkpointWritePointers = null; } - public long getWritePointer() { - return this.writePointer; + public long getTransactionId() { + return this.transactionId; } - public TTransaction setWritePointer(long writePointer) { - this.writePointer = writePointer; - setWritePointerIsSet(true); + public TTransaction setTransactionId(long transactionId) { + this.transactionId = transactionId; + setTransactionIdIsSet(true); return this; } - public void unsetWritePointer() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEPOINTER_ISSET_ID); + public void unsetTransactionId() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TRANSACTIONID_ISSET_ID); } - /** Returns true if field writePointer is set (has been assigned a value) and false otherwise */ - public boolean isSetWritePointer() { - return EncodingUtils.testBit(__isset_bitfield, __WRITEPOINTER_ISSET_ID); + /** Returns true if field transactionId is set (has been assigned a value) and false otherwise */ + public boolean isSetTransactionId() { + return EncodingUtils.testBit(__isset_bitfield, __TRANSACTIONID_ISSET_ID); } - public void setWritePointerIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEPOINTER_ISSET_ID, value); + public void setTransactionIdIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TRANSACTIONID_ISSET_ID, value); } public long getReadPointer() { @@ -454,27 +454,27 @@ public void setTypeIsSet(boolean value) { } } - public long getCurrentWritePointer() { - return this.currentWritePointer; + public long getWritePointer() { + return this.writePointer; } - public TTransaction setCurrentWritePointer(long currentWritePointer) { - this.currentWritePointer = currentWritePointer; - setCurrentWritePointerIsSet(true); + public TTransaction setWritePointer(long writePointer) { + this.writePointer = writePointer; + setWritePointerIsSet(true); return this; } - public void unsetCurrentWritePointer() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CURRENTWRITEPOINTER_ISSET_ID); + public void unsetWritePointer() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEPOINTER_ISSET_ID); } - /** Returns true if field currentWritePointer is set (has been assigned a value) and false otherwise */ - public boolean isSetCurrentWritePointer() { - return EncodingUtils.testBit(__isset_bitfield, __CURRENTWRITEPOINTER_ISSET_ID); + /** Returns true if field writePointer is set (has been assigned a value) and false otherwise */ + public boolean isSetWritePointer() { + return EncodingUtils.testBit(__isset_bitfield, __WRITEPOINTER_ISSET_ID); } - public void setCurrentWritePointerIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CURRENTWRITEPOINTER_ISSET_ID, value); + public void setWritePointerIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEPOINTER_ISSET_ID, value); } public int getCheckpointWritePointersSize() { @@ -518,11 +518,11 @@ public void setCheckpointWritePointersIsSet(boolean value) { public void setFieldValue(_Fields field, Object value) { switch (field) { - case WRITE_POINTER: + case TRANSACTION_ID: if (value == null) { - unsetWritePointer(); + unsetTransactionId(); } else { - setWritePointer((Long)value); + setTransactionId((Long)value); } break; @@ -566,11 +566,11 @@ public void setFieldValue(_Fields field, Object value) { } break; - case CURRENT_WRITE_POINTER: + case WRITE_POINTER: if (value == null) { - unsetCurrentWritePointer(); + unsetWritePointer(); } else { - setCurrentWritePointer((Long)value); + setWritePointer((Long)value); } break; @@ -587,8 +587,8 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case WRITE_POINTER: - return Long.valueOf(getWritePointer()); + case TRANSACTION_ID: + return Long.valueOf(getTransactionId()); case READ_POINTER: return Long.valueOf(getReadPointer()); @@ -605,8 +605,8 @@ public Object getFieldValue(_Fields field) { case TYPE: return getType(); - case CURRENT_WRITE_POINTER: - return Long.valueOf(getCurrentWritePointer()); + case WRITE_POINTER: + return Long.valueOf(getWritePointer()); case CHECKPOINT_WRITE_POINTERS: return getCheckpointWritePointers(); @@ -622,8 +622,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case WRITE_POINTER: - return isSetWritePointer(); + case TRANSACTION_ID: + return isSetTransactionId(); case READ_POINTER: return isSetReadPointer(); case INVALIDS: @@ -634,8 +634,8 @@ public boolean isSet(_Fields field) { return isSetFirstShort(); case TYPE: return isSetType(); - case CURRENT_WRITE_POINTER: - return isSetCurrentWritePointer(); + case WRITE_POINTER: + return isSetWritePointer(); case CHECKPOINT_WRITE_POINTERS: return isSetCheckpointWritePointers(); } @@ -655,12 +655,12 @@ public boolean equals(TTransaction that) { if (that == null) return false; - boolean this_present_writePointer = true; - boolean that_present_writePointer = true; - if (this_present_writePointer || that_present_writePointer) { - if (!(this_present_writePointer && that_present_writePointer)) + boolean this_present_transactionId = true; + boolean that_present_transactionId = true; + if (this_present_transactionId || that_present_transactionId) { + if (!(this_present_transactionId && that_present_transactionId)) return false; - if (this.writePointer != that.writePointer) + if (this.transactionId != that.transactionId) return false; } @@ -709,12 +709,12 @@ public boolean equals(TTransaction that) { return false; } - boolean this_present_currentWritePointer = true; - boolean that_present_currentWritePointer = true; - if (this_present_currentWritePointer || that_present_currentWritePointer) { - if (!(this_present_currentWritePointer && that_present_currentWritePointer)) + boolean this_present_writePointer = true; + boolean that_present_writePointer = true; + if (this_present_writePointer || that_present_writePointer) { + if (!(this_present_writePointer && that_present_writePointer)) return false; - if (this.currentWritePointer != that.currentWritePointer) + if (this.writePointer != that.writePointer) return false; } @@ -743,12 +743,12 @@ public int compareTo(TTransaction other) { int lastComparison = 0; TTransaction typedOther = (TTransaction)other; - lastComparison = Boolean.valueOf(isSetWritePointer()).compareTo(typedOther.isSetWritePointer()); + lastComparison = Boolean.valueOf(isSetTransactionId()).compareTo(typedOther.isSetTransactionId()); if (lastComparison != 0) { return lastComparison; } - if (isSetWritePointer()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writePointer, typedOther.writePointer); + if (isSetTransactionId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.transactionId, typedOther.transactionId); if (lastComparison != 0) { return lastComparison; } @@ -803,12 +803,12 @@ public int compareTo(TTransaction other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetCurrentWritePointer()).compareTo(typedOther.isSetCurrentWritePointer()); + lastComparison = Boolean.valueOf(isSetWritePointer()).compareTo(typedOther.isSetWritePointer()); if (lastComparison != 0) { return lastComparison; } - if (isSetCurrentWritePointer()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.currentWritePointer, typedOther.currentWritePointer); + if (isSetWritePointer()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writePointer, typedOther.writePointer); if (lastComparison != 0) { return lastComparison; } @@ -843,8 +843,8 @@ public String toString() { StringBuilder sb = new StringBuilder("TTransaction("); boolean first = true; - sb.append("writePointer:"); - sb.append(this.writePointer); + sb.append("transactionId:"); + sb.append(this.transactionId); first = false; if (!first) sb.append(", "); sb.append("readPointer:"); @@ -879,8 +879,8 @@ public String toString() { } first = false; if (!first) sb.append(", "); - sb.append("currentWritePointer:"); - sb.append(this.currentWritePointer); + sb.append("writePointer:"); + sb.append(this.writePointer); first = false; if (!first) sb.append(", "); sb.append("checkpointWritePointers:"); @@ -935,10 +935,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TTransaction struct break; } switch (schemeField.id) { - case 1: // WRITE_POINTER + case 1: // TRANSACTION_ID if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.writePointer = iprot.readI64(); - struct.setWritePointerIsSet(true); + struct.transactionId = iprot.readI64(); + struct.setTransactionIdIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1003,10 +1003,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TTransaction struct org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 7: // CURRENT_WRITE_POINTER + case 7: // WRITE_POINTER if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.currentWritePointer = iprot.readI64(); - struct.setCurrentWritePointerIsSet(true); + struct.writePointer = iprot.readI64(); + struct.setWritePointerIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1044,8 +1044,8 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TTransaction struc struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(WRITE_POINTER_FIELD_DESC); - oprot.writeI64(struct.writePointer); + oprot.writeFieldBegin(TRANSACTION_ID_FIELD_DESC); + oprot.writeI64(struct.transactionId); oprot.writeFieldEnd(); oprot.writeFieldBegin(READ_POINTER_FIELD_DESC); oprot.writeI64(struct.readPointer); @@ -1082,8 +1082,8 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TTransaction struc oprot.writeI32(struct.type.getValue()); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(CURRENT_WRITE_POINTER_FIELD_DESC); - oprot.writeI64(struct.currentWritePointer); + oprot.writeFieldBegin(WRITE_POINTER_FIELD_DESC); + oprot.writeI64(struct.writePointer); oprot.writeFieldEnd(); if (struct.checkpointWritePointers != null) { oprot.writeFieldBegin(CHECKPOINT_WRITE_POINTERS_FIELD_DESC); @@ -1115,7 +1115,7 @@ private static class TTransactionTupleScheme extends TupleScheme { public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.isSetWritePointer()) { + if (struct.isSetTransactionId()) { optionals.set(0); } if (struct.isSetReadPointer()) { @@ -1133,15 +1133,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct if (struct.isSetType()) { optionals.set(5); } - if (struct.isSetCurrentWritePointer()) { + if (struct.isSetWritePointer()) { optionals.set(6); } if (struct.isSetCheckpointWritePointers()) { optionals.set(7); } oprot.writeBitSet(optionals, 8); - if (struct.isSetWritePointer()) { - oprot.writeI64(struct.writePointer); + if (struct.isSetTransactionId()) { + oprot.writeI64(struct.transactionId); } if (struct.isSetReadPointer()) { oprot.writeI64(struct.readPointer); @@ -1170,8 +1170,8 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct if (struct.isSetType()) { oprot.writeI32(struct.type.getValue()); } - if (struct.isSetCurrentWritePointer()) { - oprot.writeI64(struct.currentWritePointer); + if (struct.isSetWritePointer()) { + oprot.writeI64(struct.writePointer); } if (struct.isSetCheckpointWritePointers()) { { @@ -1189,8 +1189,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(8); if (incoming.get(0)) { - struct.writePointer = iprot.readI64(); - struct.setWritePointerIsSet(true); + struct.transactionId = iprot.readI64(); + struct.setTransactionIdIsSet(true); } if (incoming.get(1)) { struct.readPointer = iprot.readI64(); @@ -1231,8 +1231,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) struct.setTypeIsSet(true); } if (incoming.get(6)) { - struct.currentWritePointer = iprot.readI64(); - struct.setCurrentWritePointerIsSet(true); + struct.writePointer = iprot.readI64(); + struct.setWritePointerIsSet(true); } if (incoming.get(7)) { { diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java index a875cb34..1687d23f 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java @@ -3370,7 +3370,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetChanges(); } else { - setChanges((Set)value); + setChanges((Set) value); } break; @@ -3861,7 +3861,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetE(); } else { - setE((TTransactionNotInProgressException)value); + setE((TTransactionNotInProgressException) value); } break; @@ -6710,7 +6710,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetE(); } else { - setE((TTransactionCouldNotTakeSnapshotException)value); + setE((TTransactionCouldNotTakeSnapshotException) value); } break; @@ -8980,7 +8980,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetTime(); } else { - setTime((Long)value); + setTime((Long) value); } break; @@ -10039,7 +10039,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetSuccess(); } else { - setSuccess((Integer)value); + setSuccess((Integer) value); } break; @@ -10795,7 +10795,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetE(); } else { - setE((TTransactionNotInProgressException)value); + setE((TTransactionNotInProgressException) value); } break; diff --git a/tephra-core/src/main/thrift/transaction.thrift b/tephra-core/src/main/thrift/transaction.thrift index 5d92b88e..6621bb5e 100644 --- a/tephra-core/src/main/thrift/transaction.thrift +++ b/tephra-core/src/main/thrift/transaction.thrift @@ -20,13 +20,13 @@ enum TTransactionType { } struct TTransaction { - 1: i64 writePointer, + 1: i64 transactionId, 2: i64 readPointer, 3: list invalids, 4: list inProgress, 5: i64 firstShort, 6: TTransactionType type, - 7: i64 currentWritePointer, + 7: i64 writePointer, 8: list checkpointWritePointers, } From 63ab1c206f9901ebce5e736fb896e9c7474708c5 Mon Sep 17 00:00:00 2001 From: Gary Helmling Date: Thu, 21 May 2015 17:12:14 -0700 Subject: [PATCH 5/5] Use VisibilityLevel setting on Transaction to control visibility of current write pointer --- .../main/java/co/cask/tephra/Transaction.java | 66 +++++++--- .../main/java/co/cask/tephra/TxConstants.java | 5 - .../TransactionConverterUtils.java | 29 +++- .../distributed/thrift/TTransaction.java | 124 +++++++++++++++++- .../thrift/TTransactionServer.java | 12 +- .../distributed/thrift/TVisibilityLevel.java | 61 +++++++++ .../src/main/thrift/transaction.thrift | 6 + .../coprocessor/TransactionProcessor.java | 21 +-- .../TransactionVisibilityFilter.java | 24 +--- .../hbase96/TransactionAwareHTableTest.java | 6 +- .../coprocessor/TransactionProcessor.java | 21 +-- .../TransactionVisibilityFilter.java | 24 +--- .../hbase98/TransactionAwareHTableTest.java | 6 +- 13 files changed, 284 insertions(+), 121 deletions(-) create mode 100644 tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TVisibilityLevel.java diff --git a/tephra-api/src/main/java/co/cask/tephra/Transaction.java b/tephra-api/src/main/java/co/cask/tephra/Transaction.java index c7315f49..a80ef3a9 100644 --- a/tephra-api/src/main/java/co/cask/tephra/Transaction.java +++ b/tephra-api/src/main/java/co/cask/tephra/Transaction.java @@ -33,12 +33,36 @@ public class Transaction { private final TransactionType type; private final long[] checkpointWritePointers; + private VisibilityLevel visibilityLevel = VisibilityLevel.SNAPSHOT; + private static final long[] NO_EXCLUDES = { }; public static final long NO_TX_IN_PROGRESS = Long.MAX_VALUE; public static final Transaction ALL_VISIBLE_LATEST = new Transaction(Long.MAX_VALUE, Long.MAX_VALUE, NO_EXCLUDES, NO_EXCLUDES, NO_TX_IN_PROGRESS, TransactionType.SHORT); + /** + * Defines the possible visibility levels for read operations. + * + *

+ *

    + *
  • SNAPSHOT - uses the transaction's read snapshot, plus includes all write pointers from the + * current transaction
  • + *
  • SNAPSHOT_EXCLUDE_CURRENT - uses the transaction's read snapshot, plus includes all write + * pointers from the current transaction, except the current write pointer + * (see {@link #getWritePointer()})
  • + *
+ *

+ * + *

The default value used is {@code SNAPSHOT}.

+ * + * @see #setVisibility(VisibilityLevel) + */ + public enum VisibilityLevel { + SNAPSHOT, + SNAPSHOT_EXCLUDE_CURRENT + } + /** * Creates a new short transaction. * @param readPointer read pointer for transaction @@ -63,7 +87,8 @@ public Transaction(long readPointer, long txId, long[] invalids, long[] inProgre */ public Transaction(long readPointer, long txId, long[] invalids, long[] inProgress, long firstShortInProgress, TransactionType type) { - this(readPointer, txId, txId, invalids, inProgress, firstShortInProgress, type, new long[0]); + this(readPointer, txId, txId, invalids, inProgress, firstShortInProgress, type, new long[0], + VisibilityLevel.SNAPSHOT); } /** @@ -78,9 +103,11 @@ public Transaction(long readPointer, long txId, long[] invalids, long[] inProgre * @param firstShortInProgress earliest in-progress short transaction * @param type transaction type * @param checkpointPointers the list of writer pointers added from checkpoints on the transaction + * @param visibilityLevel the visibility level to use for transactional reads */ public Transaction(long readPointer, long txId, long writePointer, long[] invalids, long[] inProgress, - long firstShortInProgress, TransactionType type, long[] checkpointPointers) { + long firstShortInProgress, TransactionType type, long[] checkpointPointers, + VisibilityLevel visibilityLevel) { this.readPointer = readPointer; this.txId = txId; this.writePointer = writePointer; @@ -89,6 +116,7 @@ public Transaction(long readPointer, long txId, long writePointer, long[] invali this.firstShortInProgress = firstShortInProgress; this.type = type; this.checkpointWritePointers = checkpointPointers; + this.visibilityLevel = visibilityLevel; } /** @@ -101,7 +129,8 @@ public Transaction(long readPointer, long txId, long writePointer, long[] invali */ public Transaction(Transaction toCopy, long writePointer, long[] checkpointPointers) { this(toCopy.getReadPointer(), toCopy.getTransactionId(), writePointer, toCopy.getInvalids(), - toCopy.getInProgress(), toCopy.getFirstShortInProgress(), toCopy.getType(), checkpointPointers); + toCopy.getInProgress(), toCopy.getFirstShortInProgress(), toCopy.getType(), checkpointPointers, + toCopy.getVisibilityLevel()); } public long getReadPointer() { @@ -190,27 +219,27 @@ public boolean isCheckpoint(long version) { * @param version the data version to check for visibility * @return true if the version is visible, false if it should be hidden (filtered) * - * @see #isVisible(long, boolean) to exclude the current write pointer from visible versions. This method always - * includes the current write pointer. + * @see #setVisibility(VisibilityLevel) to control whether the current write pointer is visible. */ public boolean isVisible(long version) { - return isVisible(version, true); + // either it was committed before or the change belongs to current tx + return (version <= getReadPointer() && !isExcluded(version)) || + ((txId == version || isCheckpoint(version)) && + (visibilityLevel == VisibilityLevel.SNAPSHOT || writePointer != version)); } /** - * Returns whether or not the given version should be visible to the current transaction. A version will be visible - * if it was successfully committed prior to the current transaction starting, or was written by the current - * transaction (using either the current write pointer or the write pointer from a prior checkpoint). - * - * @param version the data version to check for visibility - * @param excludeCurrentWritePointer whether writes from the current write pointer should be visible - * @return true if the version is visible, false if it should be hidden (filtered) + * Sets the visibility level for read operations. */ - public boolean isVisible(long version, boolean excludeCurrentWritePointer) { - // either it was committed before or the change belongs to current tx - return (version <= getReadPointer() && !isExcluded(version)) || - ((txId == version || isCheckpoint(version)) && - (!excludeCurrentWritePointer || writePointer != version)); + public void setVisibility(VisibilityLevel level) { + this.visibilityLevel = level; + } + + /** + * Returns the currently set visibility level. + */ + public VisibilityLevel getVisibilityLevel() { + return visibilityLevel; } public boolean hasExcludes() { @@ -244,6 +273,7 @@ public String toString() { .append(", firstShortInProgress: ").append(firstShortInProgress) .append(", type: ").append(type) .append(", checkpointWritePointers: ").append(Arrays.toString(checkpointWritePointers)) + .append(", visibilityLevel: ").append(visibilityLevel) .append('}') .toString(); } diff --git a/tephra-core/src/main/java/co/cask/tephra/TxConstants.java b/tephra-core/src/main/java/co/cask/tephra/TxConstants.java index e26f585b..d45ad28f 100644 --- a/tephra-core/src/main/java/co/cask/tephra/TxConstants.java +++ b/tephra-core/src/main/java/co/cask/tephra/TxConstants.java @@ -97,11 +97,6 @@ public enum ConflictDetection { */ public static final byte[] FAMILY_DELETE_QUALIFIER = new byte[0]; - /** - * Attribute key used to exclude the transaction's current write pointer from reads. - */ - public static final String TX_EXCLUDE_CURRENT_WRITE = "cask.tx.exclude.current"; - // Constants for monitoring status public static final String STATUS_OK = "OK"; public static final String STATUS_NOTOK = "NOTOK"; diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java index ecc79f19..070b43f0 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/TransactionConverterUtils.java @@ -20,6 +20,7 @@ import co.cask.tephra.TransactionType; import co.cask.tephra.distributed.thrift.TTransaction; import co.cask.tephra.distributed.thrift.TTransactionType; +import co.cask.tephra.distributed.thrift.TVisibilityLevel; import com.google.common.primitives.Longs; /** @@ -31,14 +32,16 @@ public static TTransaction wrap(Transaction tx) { return new TTransaction(tx.getTransactionId(), tx.getReadPointer(), Longs.asList(tx.getInvalids()), Longs.asList(tx.getInProgress()), tx.getFirstShortInProgress(), getTTransactionType(tx.getType()), - tx.getWritePointer(), Longs.asList(tx.getCheckpointWritePointers())); + tx.getWritePointer(), Longs.asList(tx.getCheckpointWritePointers()), + getTVisibilityLevel(tx.getVisibilityLevel())); } public static Transaction unwrap(TTransaction thriftTx) { return new Transaction(thriftTx.getReadPointer(), thriftTx.getTransactionId(), thriftTx.getWritePointer(), Longs.toArray(thriftTx.getInvalids()), Longs.toArray(thriftTx.getInProgress()), thriftTx.getFirstShort(), getTransactionType(thriftTx.getType()), - Longs.toArray(thriftTx.getCheckpointWritePointers())); + Longs.toArray(thriftTx.getCheckpointWritePointers()), + getVisibilityLevel(thriftTx.getVisibilityLevel())); } private static TransactionType getTransactionType(TTransactionType tType) { @@ -48,4 +51,26 @@ private static TransactionType getTransactionType(TTransactionType tType) { private static TTransactionType getTTransactionType(TransactionType type) { return type == TransactionType.SHORT ? TTransactionType.SHORT : TTransactionType.LONG; } + + private static Transaction.VisibilityLevel getVisibilityLevel(TVisibilityLevel tLevel) { + switch (tLevel) { + case SNAPSHOT: + return Transaction.VisibilityLevel.SNAPSHOT; + case SNAPSHOT_EXCLUDE_CURRENT: + return Transaction.VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT; + default: + throw new IllegalArgumentException("Unknown TVisibilityLevel: " + tLevel); + } + } + + private static TVisibilityLevel getTVisibilityLevel(Transaction.VisibilityLevel level) { + switch (level) { + case SNAPSHOT: + return TVisibilityLevel.SNAPSHOT; + case SNAPSHOT_EXCLUDE_CURRENT: + return TVisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT; + default: + throw new IllegalArgumentException("Unknown VisibilityLevel: " + level); + } + } } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java index 068e8e26..32474ba8 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransaction.java @@ -57,6 +57,7 @@ public class TTransaction implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -76,6 +77,11 @@ public class TTransaction implements org.apache.thrift.TBase checkpointWritePointers; // required + /** + * + * @see TVisibilityLevel + */ + public TVisibilityLevel visibilityLevel; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -90,7 +96,12 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ TYPE((short)6, "type"), WRITE_POINTER((short)7, "writePointer"), - CHECKPOINT_WRITE_POINTERS((short)8, "checkpointWritePointers"); + CHECKPOINT_WRITE_POINTERS((short)8, "checkpointWritePointers"), + /** + * + * @see TVisibilityLevel + */ + VISIBILITY_LEVEL((short)9, "visibilityLevel"); private static final Map byName = new HashMap(); @@ -121,6 +132,8 @@ public static _Fields findByThriftId(int fieldId) { return WRITE_POINTER; case 8: // CHECKPOINT_WRITE_POINTERS return CHECKPOINT_WRITE_POINTERS; + case 9: // VISIBILITY_LEVEL + return VISIBILITY_LEVEL; default: return null; } @@ -188,6 +201,8 @@ public String getFieldName() { tmpMap.put(_Fields.CHECKPOINT_WRITE_POINTERS, new org.apache.thrift.meta_data.FieldMetaData("checkpointWritePointers", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + tmpMap.put(_Fields.VISIBILITY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("visibilityLevel", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TVisibilityLevel.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTransaction.class, metaDataMap); } @@ -203,7 +218,8 @@ public TTransaction( long firstShort, TTransactionType type, long writePointer, - List checkpointWritePointers) + List checkpointWritePointers, + TVisibilityLevel visibilityLevel) { this(); this.transactionId = transactionId; @@ -218,6 +234,7 @@ public TTransaction( this.writePointer = writePointer; setWritePointerIsSet(true); this.checkpointWritePointers = checkpointWritePointers; + this.visibilityLevel = visibilityLevel; } /** @@ -253,6 +270,9 @@ public TTransaction(TTransaction other) { } this.checkpointWritePointers = __this__checkpointWritePointers; } + if (other.isSetVisibilityLevel()) { + this.visibilityLevel = other.visibilityLevel; + } } public TTransaction deepCopy() { @@ -273,6 +293,7 @@ public void clear() { setWritePointerIsSet(false); this.writePointer = 0; this.checkpointWritePointers = null; + this.visibilityLevel = null; } public long getTransactionId() { @@ -516,6 +537,38 @@ public void setCheckpointWritePointersIsSet(boolean value) { } } + /** + * + * @see TVisibilityLevel + */ + public TVisibilityLevel getVisibilityLevel() { + return this.visibilityLevel; + } + + /** + * + * @see TVisibilityLevel + */ + public TTransaction setVisibilityLevel(TVisibilityLevel visibilityLevel) { + this.visibilityLevel = visibilityLevel; + return this; + } + + public void unsetVisibilityLevel() { + this.visibilityLevel = null; + } + + /** Returns true if field visibilityLevel is set (has been assigned a value) and false otherwise */ + public boolean isSetVisibilityLevel() { + return this.visibilityLevel != null; + } + + public void setVisibilityLevelIsSet(boolean value) { + if (!value) { + this.visibilityLevel = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case TRANSACTION_ID: @@ -582,6 +635,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case VISIBILITY_LEVEL: + if (value == null) { + unsetVisibilityLevel(); + } else { + setVisibilityLevel((TVisibilityLevel)value); + } + break; + } } @@ -611,6 +672,9 @@ public Object getFieldValue(_Fields field) { case CHECKPOINT_WRITE_POINTERS: return getCheckpointWritePointers(); + case VISIBILITY_LEVEL: + return getVisibilityLevel(); + } throw new IllegalStateException(); } @@ -638,6 +702,8 @@ public boolean isSet(_Fields field) { return isSetWritePointer(); case CHECKPOINT_WRITE_POINTERS: return isSetCheckpointWritePointers(); + case VISIBILITY_LEVEL: + return isSetVisibilityLevel(); } throw new IllegalStateException(); } @@ -727,6 +793,15 @@ public boolean equals(TTransaction that) { return false; } + boolean this_present_visibilityLevel = true && this.isSetVisibilityLevel(); + boolean that_present_visibilityLevel = true && that.isSetVisibilityLevel(); + if (this_present_visibilityLevel || that_present_visibilityLevel) { + if (!(this_present_visibilityLevel && that_present_visibilityLevel)) + return false; + if (!this.visibilityLevel.equals(that.visibilityLevel)) + return false; + } + return true; } @@ -823,6 +898,16 @@ public int compareTo(TTransaction other) { return lastComparison; } } + lastComparison = Boolean.valueOf(isSetVisibilityLevel()).compareTo(typedOther.isSetVisibilityLevel()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetVisibilityLevel()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.visibilityLevel, typedOther.visibilityLevel); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -890,6 +975,14 @@ public String toString() { sb.append(this.checkpointWritePointers); } first = false; + if (!first) sb.append(", "); + sb.append("visibilityLevel:"); + if (this.visibilityLevel == null) { + sb.append("null"); + } else { + sb.append(this.visibilityLevel); + } + first = false; sb.append(")"); return sb.toString(); } @@ -1029,6 +1122,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TTransaction struct org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 9: // VISIBILITY_LEVEL + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.visibilityLevel = TVisibilityLevel.findByValue(iprot.readI32()); + struct.setVisibilityLevelIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1097,6 +1198,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TTransaction struc } oprot.writeFieldEnd(); } + if (struct.visibilityLevel != null) { + oprot.writeFieldBegin(VISIBILITY_LEVEL_FIELD_DESC); + oprot.writeI32(struct.visibilityLevel.getValue()); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1139,7 +1245,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct if (struct.isSetCheckpointWritePointers()) { optionals.set(7); } - oprot.writeBitSet(optionals, 8); + if (struct.isSetVisibilityLevel()) { + optionals.set(8); + } + oprot.writeBitSet(optionals, 9); if (struct.isSetTransactionId()) { oprot.writeI64(struct.transactionId); } @@ -1182,12 +1291,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TTransaction struct } } } + if (struct.isSetVisibilityLevel()) { + oprot.writeI32(struct.visibilityLevel.getValue()); + } } @Override public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(8); + BitSet incoming = iprot.readBitSet(9); if (incoming.get(0)) { struct.transactionId = iprot.readI64(); struct.setTransactionIdIsSet(true); @@ -1247,6 +1359,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TTransaction struct) } struct.setCheckpointWritePointersIsSet(true); } + if (incoming.get(8)) { + struct.visibilityLevel = TVisibilityLevel.findByValue(iprot.readI32()); + struct.setVisibilityLevelIsSet(true); + } } } diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java index 1687d23f..a875cb34 100644 --- a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TTransactionServer.java @@ -3370,7 +3370,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetChanges(); } else { - setChanges((Set) value); + setChanges((Set)value); } break; @@ -3861,7 +3861,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetE(); } else { - setE((TTransactionNotInProgressException) value); + setE((TTransactionNotInProgressException)value); } break; @@ -6710,7 +6710,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetE(); } else { - setE((TTransactionCouldNotTakeSnapshotException) value); + setE((TTransactionCouldNotTakeSnapshotException)value); } break; @@ -8980,7 +8980,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetTime(); } else { - setTime((Long) value); + setTime((Long)value); } break; @@ -10039,7 +10039,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetSuccess(); } else { - setSuccess((Integer) value); + setSuccess((Integer)value); } break; @@ -10795,7 +10795,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetE(); } else { - setE((TTransactionNotInProgressException) value); + setE((TTransactionNotInProgressException)value); } break; diff --git a/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TVisibilityLevel.java b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TVisibilityLevel.java new file mode 100644 index 00000000..878f4b48 --- /dev/null +++ b/tephra-core/src/main/java/co/cask/tephra/distributed/thrift/TVisibilityLevel.java @@ -0,0 +1,61 @@ +/* + * Copyright © 2015 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package co.cask.tephra.distributed.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TVisibilityLevel implements org.apache.thrift.TEnum { + SNAPSHOT(1), + SNAPSHOT_EXCLUDE_CURRENT(2); + + private final int value; + + private TVisibilityLevel(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TVisibilityLevel findByValue(int value) { + switch (value) { + case 1: + return SNAPSHOT; + case 2: + return SNAPSHOT_EXCLUDE_CURRENT; + default: + return null; + } + } +} diff --git a/tephra-core/src/main/thrift/transaction.thrift b/tephra-core/src/main/thrift/transaction.thrift index 6621bb5e..b05a42c0 100644 --- a/tephra-core/src/main/thrift/transaction.thrift +++ b/tephra-core/src/main/thrift/transaction.thrift @@ -19,6 +19,11 @@ enum TTransactionType { LONG = 2 } +enum TVisibilityLevel { + SNAPSHOT = 1, + SNAPSHOT_EXCLUDE_CURRENT = 2 +} + struct TTransaction { 1: i64 transactionId, 2: i64 readPointer, @@ -28,6 +33,7 @@ struct TTransaction { 6: TTransactionType type, 7: i64 writePointer, 8: list checkpointWritePointers, + 9: TVisibilityLevel visibilityLevel } exception TTransactionNotInProgressException { diff --git a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java index ea820130..f3b3990b 100644 --- a/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java +++ b/tephra-hbase-compat-0.96/src/main/java/co/cask/tephra/hbase96/coprocessor/TransactionProcessor.java @@ -149,13 +149,10 @@ public void preGetOp(ObserverContext e, Get get, L throws IOException { Transaction tx = getFromOperation(get); if (tx != null) { - boolean excludeCurrentWritePtr = isAttributeSet(get, TxConstants.TX_EXCLUDE_CURRENT_WRITE); - projectFamilyDeletes(get); get.setMaxVersions(); get.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx), TxUtils.getMaxVisibleTimestamp(tx)); - Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN, excludeCurrentWritePtr), - get.getFilter()); + Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN), get.getFilter()); get.setFilter(newFilter); } } @@ -203,13 +200,10 @@ public RegionScanner preScannerOpen(ObserverContext ttlByFamily */ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, ScanType scanType, @Nullable Filter cellFilter) { - this(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter, false); - } - - /** - * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. - * - * @param tx the current transaction to apply. Only data visible to this transaction will be returned. - * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name - * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} - * these will be interpreted as "delete" markers and the column will be filtered out - * @param scanType the type of scan operation being performed - * @param cellFilter if non-null, this filter will be applied to all cells visible to the current transaction, by - * calling {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}. If null, then - * {@link Filter.ReturnCode#INCLUDE_AND_NEXT_COL} will be returned instead. - * @param excludeCurrentWritePtr if true, cells written with the transaction's current write pointer will be filtered - * out from results returned by this filter. - */ - public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, - ScanType scanType, @Nullable Filter cellFilter, boolean excludeCurrentWritePtr) { this.tx = tx; this.oldestTsByFamily = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); for (Map.Entry ttlEntry : ttlByFamily.entrySet()) { @@ -114,7 +93,6 @@ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily this.clearDeletes = scanType == ScanType.COMPACT_DROP_DELETES || scanType == ScanType.USER_SCAN; this.cellFilter = cellFilter; - this.excludeCurrentWritePtr = excludeCurrentWritePtr; } @Override @@ -131,7 +109,7 @@ public ReturnCode filterKeyValue(Cell cell) throws IOException { if (kvTimestamp < currentOldestTs) { // passed TTL for this column, seek to next return ReturnCode.NEXT_COL; - } else if (tx.isVisible(kvTimestamp, excludeCurrentWritePtr)) { + } else if (tx.isVisible(kvTimestamp)) { if (deleteTracker.isFamilyDelete(cell)) { deleteTracker.addFamilyDelete(cell); if (clearDeletes) { diff --git a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java index 066d5185..705f88b6 100644 --- a/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java +++ b/tephra-hbase-compat-0.96/src/test/java/co/cask/tephra/hbase96/TransactionAwareHTableTest.java @@ -792,19 +792,16 @@ public void testCheckpoint() throws Exception { verifyRow(transactionAwareHTable, TestBytes.row3, TestBytes.value); // when disabling current write pointer, only the previous checkpoints should be visible + transactionContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT); Get get = new Get(TestBytes.row); - get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); verifyRow(transactionAwareHTable, get, TestBytes.value); get = new Get(TestBytes.row2); - get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); verifyRow(transactionAwareHTable, get, TestBytes.value2); get = new Get(TestBytes.row3); - get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); verifyRow(transactionAwareHTable, get, null); // test scan results excluding current write pointer Scan scan = new Scan(); - scan.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); ResultScanner scanner = transactionAwareHTable.getScanner(scan); Result row = scanner.next(); @@ -822,6 +819,7 @@ public void testCheckpoint() throws Exception { row = scanner.next(); assertNull(row); scanner.close(); + transactionContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT); // check that writes are still not visible to other clients TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table)); diff --git a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java index bae143a3..8478141f 100644 --- a/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java +++ b/tephra-hbase-compat-0.98/src/main/java/co/cask/tephra/hbase98/coprocessor/TransactionProcessor.java @@ -149,13 +149,10 @@ public void preGetOp(ObserverContext e, Get get, L throws IOException { Transaction tx = getFromOperation(get); if (tx != null) { - boolean excludeCurrentWritePtr = isAttributeSet(get, TxConstants.TX_EXCLUDE_CURRENT_WRITE); - projectFamilyDeletes(get); get.setMaxVersions(); get.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx), TxUtils.getMaxVisibleTimestamp(tx)); - Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN, excludeCurrentWritePtr), - get.getFilter()); + Filter newFilter = Filters.combine(getTransactionFilter(tx, ScanType.USER_SCAN), get.getFilter()); get.setFilter(newFilter); } } @@ -203,13 +200,10 @@ public RegionScanner preScannerOpen(ObserverContext ttlByFamily */ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, ScanType scanType, @Nullable Filter cellFilter) { - this(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter, false); - } - - /** - * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. - * - * @param tx the current transaction to apply. Only data visible to this transaction will be returned. - * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name - * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} - * these will be interpreted as "delete" markers and the column will be filtered out - * @param scanType the type of scan operation being performed - * @param cellFilter if non-null, this filter will be applied to all cells visible to the current transaction, by - * calling {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}. If null, then - * {@link Filter.ReturnCode#INCLUDE_AND_NEXT_COL} will be returned instead. - * @param excludeCurrentWritePtr if true, cells written with the transaction's current write pointer will be filtered - * out from results returned by this filter. - */ - public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, - ScanType scanType, @Nullable Filter cellFilter, boolean excludeCurrentWritePtr) { this.tx = tx; this.oldestTsByFamily = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); for (Map.Entry ttlEntry : ttlByFamily.entrySet()) { @@ -114,7 +93,6 @@ public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily this.clearDeletes = scanType == ScanType.COMPACT_DROP_DELETES || scanType == ScanType.USER_SCAN; this.cellFilter = cellFilter; - this.excludeCurrentWritePtr = excludeCurrentWritePtr; } @Override @@ -131,7 +109,7 @@ public ReturnCode filterKeyValue(Cell cell) throws IOException { if (kvTimestamp < currentOldestTs) { // passed TTL for this column, seek to next return ReturnCode.NEXT_COL; - } else if (tx.isVisible(kvTimestamp, excludeCurrentWritePtr)) { + } else if (tx.isVisible(kvTimestamp)) { if (deleteTracker.isFamilyDelete(cell)) { deleteTracker.addFamilyDelete(cell); if (clearDeletes) { diff --git a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java index db154764..78275051 100644 --- a/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java +++ b/tephra-hbase-compat-0.98/src/test/java/co/cask/tephra/hbase98/TransactionAwareHTableTest.java @@ -793,19 +793,16 @@ public void testCheckpoint() throws Exception { verifyRow(transactionAwareHTable, TestBytes.row3, TestBytes.value); // when disabling current write pointer, only the previous checkpoints should be visible + transactionContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT); Get get = new Get(TestBytes.row); - get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); verifyRow(transactionAwareHTable, get, TestBytes.value); get = new Get(TestBytes.row2); - get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); verifyRow(transactionAwareHTable, get, TestBytes.value2); get = new Get(TestBytes.row3); - get.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); verifyRow(transactionAwareHTable, get, null); // test scan results excluding current write pointer Scan scan = new Scan(); - scan.setAttribute(TxConstants.TX_EXCLUDE_CURRENT_WRITE, new byte[]{ (byte) 1 }); ResultScanner scanner = transactionAwareHTable.getScanner(scan); Result row = scanner.next(); @@ -823,6 +820,7 @@ public void testCheckpoint() throws Exception { row = scanner.next(); assertNull(row); scanner.close(); + transactionContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT); // check that writes are still not visible to other clients TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table));