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));