You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tephra.apache.org by an...@apache.org on 2017/09/06 08:01:12 UTC

[1/5] incubator-tephra git commit: Support for HBase 1.3.x

Repository: incubator-tephra
Updated Branches:
  refs/heads/master fd6ef73d3 -> 8f958edb6


http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java
new file mode 100644
index 0000000..ac5e923
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java
@@ -0,0 +1,459 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.TransactionContext;
+import org.apache.tephra.TransactionManager;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.coprocessor.CacheSupplier;
+import org.apache.tephra.coprocessor.TransactionStateCache;
+import org.apache.tephra.hbase.AbstractHBaseTableTest;
+import org.apache.tephra.hbase.TransactionAwareHTable;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+import org.apache.tephra.inmemory.InMemoryTxSystemClient;
+import org.apache.tephra.metrics.TxMetricsCollector;
+import org.apache.tephra.persist.InMemoryTransactionStateStorage;
+import org.apache.tephra.persist.TransactionSnapshot;
+import org.apache.tephra.persist.TransactionStateStorage;
+import org.apache.tephra.persist.TransactionVisibilityState;
+import org.apache.tephra.txprune.TransactionPruningPlugin;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Test invalid list pruning
+ */
+public class InvalidListPruneTest extends AbstractHBaseTableTest {
+  private static final byte[] family = Bytes.toBytes("f1");
+  private static final byte[] qualifier = Bytes.toBytes("col1");
+  private static final int MAX_ROWS = 1000;
+
+  private static TableName txDataTable1;
+  private static TableName pruneStateTable;
+  private static DataJanitorState dataJanitorState;
+
+  // Override AbstractHBaseTableTest.startMiniCluster to setup configuration
+  @BeforeClass
+  public static void startMiniCluster() throws Exception {
+    // Setup the configuration to start HBase cluster with the invalid list pruning enabled
+    conf = HBaseConfiguration.create();
+    conf.setBoolean(TxConstants.TransactionPruning.PRUNE_ENABLE, true);
+    // Flush prune data to table quickly, so that tests don't need have to wait long to see updates
+    conf.setLong(TxConstants.TransactionPruning.PRUNE_FLUSH_INTERVAL, 0L);
+    AbstractHBaseTableTest.startMiniCluster();
+
+    TransactionStateStorage txStateStorage = new InMemoryTransactionStateStorage();
+    TransactionManager txManager = new TransactionManager(conf, txStateStorage, new TxMetricsCollector());
+    txManager.startAndWait();
+
+    // Do some transactional data operations
+    txDataTable1 = TableName.valueOf("invalidListPruneTestTable1");
+    HTable hTable = createTable(txDataTable1.getName(), new byte[][]{family}, false,
+                                Collections.singletonList(TestTransactionProcessor.class.getName()));
+    try (TransactionAwareHTable txTable = new TransactionAwareHTable(hTable, TxConstants.ConflictDetection.ROW)) {
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+      txContext.start();
+      for (int i = 0; i < MAX_ROWS; ++i) {
+        txTable.put(new Put(Bytes.toBytes(i)).addColumn(family, qualifier, Bytes.toBytes(i)));
+      }
+      txContext.finish();
+    }
+
+    testUtil.flush(txDataTable1);
+    txManager.stopAndWait();
+
+    pruneStateTable = TableName.valueOf(conf.get(TxConstants.TransactionPruning.PRUNE_STATE_TABLE,
+                                                 TxConstants.TransactionPruning.DEFAULT_PRUNE_STATE_TABLE));
+    dataJanitorState =
+      new DataJanitorState(new DataJanitorState.TableSupplier() {
+        @Override
+        public Table get() throws IOException {
+          return testUtil.getConnection().getTable(pruneStateTable);
+        }
+      });
+
+  }
+
+  @AfterClass
+  public static void shutdownAfterClass() throws Exception {
+    hBaseAdmin.disableTable(txDataTable1);
+    hBaseAdmin.deleteTable(txDataTable1);
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    createPruneStateTable();
+    InMemoryTransactionStateCache.setTransactionSnapshot(null);
+  }
+
+  private void createPruneStateTable() throws Exception {
+    HTable table = createTable(pruneStateTable.getName(), new byte[][]{DataJanitorState.FAMILY}, false,
+                               // Prune state table is a non-transactional table, hence no transaction co-processor
+                               Collections.<String>emptyList());
+    table.close();
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    // Disable the data table so that prune writer thread gets stopped,
+    // this makes sure that any cached value will not interfere with next test
+    hBaseAdmin.disableTable(txDataTable1);
+    deletePruneStateTable();
+    // Enabling the table enables the prune writer thread again
+    hBaseAdmin.enableTable(txDataTable1);
+  }
+
+  private void deletePruneStateTable() throws Exception {
+    if (hBaseAdmin.tableExists(pruneStateTable)) {
+      hBaseAdmin.disableTable(pruneStateTable);
+      hBaseAdmin.deleteTable(pruneStateTable);
+    }
+  }
+
+  @Test
+  public void testRecordCompactionState() throws Exception {
+    // No prune upper bound initially
+    Assert.assertEquals(-1,
+                        dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+
+    // Create a new transaction snapshot
+    InMemoryTransactionStateCache.setTransactionSnapshot(
+      new TransactionSnapshot(100, 100, 100, ImmutableSet.of(50L),
+                              ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
+    // Run minor compaction
+    testUtil.compact(txDataTable1, false);
+    // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+    TimeUnit.SECONDS.sleep(2);
+    // No prune upper bound after minor compaction too
+    Assert.assertEquals(-1,
+                        dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+
+    // Run major compaction, and verify prune upper bound
+    testUtil.compact(txDataTable1, true);
+    // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+    TimeUnit.SECONDS.sleep(2);
+    Assert.assertEquals(50,
+                        dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+
+    // Run major compaction again with same snapshot, prune upper bound should not change
+    testUtil.compact(txDataTable1, true);
+    // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+    TimeUnit.SECONDS.sleep(2);
+    Assert.assertEquals(50,
+                        dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+
+    // Create a new transaction snapshot
+    InMemoryTransactionStateCache.setTransactionSnapshot(
+      new TransactionSnapshot(110, 111, 112, ImmutableSet.of(150L),
+                              ImmutableSortedMap.of(105L, new TransactionManager.InProgressTx(
+                                100, 30, TransactionManager.InProgressType.SHORT))));
+    Assert.assertEquals(50,
+                        dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+
+    // Run major compaction again, now prune upper bound should change
+    testUtil.compact(txDataTable1, true);
+    // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+    TimeUnit.SECONDS.sleep(2);
+    Assert.assertEquals(104,
+                        dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+  }
+
+  @Test
+  public void testRecordCompactionStateNoTable() throws Exception {
+    // Create a new transaction snapshot
+    InMemoryTransactionStateCache.setTransactionSnapshot(
+      new TransactionSnapshot(100, 100, 100, ImmutableSet.of(50L),
+                              ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
+    // Run major compaction, and verify it completes
+    long now = System.currentTimeMillis();
+    testUtil.compact(txDataTable1, true);
+    // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+    TimeUnit.SECONDS.sleep(2);
+    long lastMajorCompactionTime = TestTransactionProcessor.lastMajorCompactionTime.get();
+    Assert.assertTrue(String.format("Expected %d, but was %d", now, lastMajorCompactionTime),
+                      lastMajorCompactionTime >= now);
+  }
+
+  @Test
+  public void testRecordCompactionStateNoTxSnapshot() throws Exception {
+    // Test recording state without having a transaction snapshot to make sure we don't disrupt
+    // major compaction in that case
+    InMemoryTransactionStateCache.setTransactionSnapshot(null);
+    // Run major compaction, and verify it completes
+    long now = System.currentTimeMillis();
+    testUtil.compact(txDataTable1, true);
+    // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+    TimeUnit.SECONDS.sleep(2);
+    long lastMajorCompactionTime = TestTransactionProcessor.lastMajorCompactionTime.get();
+    Assert.assertTrue(String.format("Expected %d, but was %d", now, lastMajorCompactionTime),
+                      lastMajorCompactionTime >= now);
+  }
+
+  @Test
+  public void testPruneUpperBound() throws Exception {
+    TransactionPruningPlugin transactionPruningPlugin = new TestTransactionPruningPlugin();
+    transactionPruningPlugin.initialize(conf);
+
+    try {
+      // Run without a transaction snapshot first
+      long now1 = 200;
+      long inactiveTxTimeNow1 = 150 * TxConstants.MAX_TX_PER_MS;
+      long expectedPruneUpperBound1 = -1;
+      // fetch prune upper bound
+      long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
+      Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound1);
+
+      TimeRegions expectedRegions1 =
+        new TimeRegions(now1,
+                        ImmutableSortedSet.orderedBy(Bytes.BYTES_COMPARATOR)
+                          .add(getRegionName(txDataTable1, Bytes.toBytes(0)))
+                          .build());
+      // Assert prune state is recorded correctly
+      Assert.assertEquals(expectedRegions1, dataJanitorState.getRegionsOnOrBeforeTime(now1));
+      Assert.assertEquals(expectedPruneUpperBound1,
+                          dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+      Assert.assertEquals(inactiveTxTimeNow1, dataJanitorState.getInactiveTransactionBoundForTime(now1));
+
+      // Run prune complete
+      transactionPruningPlugin.pruneComplete(now1, expectedPruneUpperBound1);
+
+      // Assert prune state was cleaned up correctly based on the prune time
+      Assert.assertEquals(expectedRegions1, dataJanitorState.getRegionsOnOrBeforeTime(now1));
+      Assert.assertEquals(expectedPruneUpperBound1,
+                          dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+      Assert.assertEquals(inactiveTxTimeNow1, dataJanitorState.getInactiveTransactionBoundForTime(now1));
+
+      // Create a new transaction snapshot, and run major compaction on txDataTable1
+      // And run all assertions again
+      long now2 = 300;
+      long inactiveTxTimeNow2 = 250 * TxConstants.MAX_TX_PER_MS;
+      long expectedPruneUpperBound2 = 200 * TxConstants.MAX_TX_PER_MS;
+      InMemoryTransactionStateCache.setTransactionSnapshot(
+        new TransactionSnapshot(expectedPruneUpperBound2, expectedPruneUpperBound2, expectedPruneUpperBound2,
+                                ImmutableSet.of(expectedPruneUpperBound2),
+                                ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
+      TimeRegions expectedRegions2 =
+        new TimeRegions(now2,
+                        ImmutableSortedSet.orderedBy(Bytes.BYTES_COMPARATOR)
+                          .add(getRegionName(txDataTable1, Bytes.toBytes(0)))
+                          .build());
+      testUtil.compact(txDataTable1, true);
+      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+      TimeUnit.SECONDS.sleep(2);
+      long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
+      Assert.assertEquals(expectedPruneUpperBound2, pruneUpperBound2);
+
+      Assert.assertEquals(expectedRegions2, dataJanitorState.getRegionsOnOrBeforeTime(now2));
+      Assert.assertEquals(expectedPruneUpperBound2,
+                          dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+      Assert.assertEquals(inactiveTxTimeNow2, dataJanitorState.getInactiveTransactionBoundForTime(now2));
+      Assert.assertEquals(expectedRegions1, dataJanitorState.getRegionsOnOrBeforeTime(now1));
+      Assert.assertEquals(inactiveTxTimeNow1, dataJanitorState.getInactiveTransactionBoundForTime(now1));
+
+      transactionPruningPlugin.pruneComplete(now2, pruneUpperBound2);
+      Assert.assertEquals(expectedRegions2, dataJanitorState.getRegionsOnOrBeforeTime(now2));
+      Assert.assertEquals(expectedPruneUpperBound2,
+                          dataJanitorState.getPruneUpperBoundForRegion(getRegionName(txDataTable1, Bytes.toBytes(0))));
+      Assert.assertEquals(inactiveTxTimeNow2, dataJanitorState.getInactiveTransactionBoundForTime(now2));
+      Assert.assertNull(dataJanitorState.getRegionsOnOrBeforeTime(now1));
+      Assert.assertEquals(expectedPruneUpperBound1, dataJanitorState.getInactiveTransactionBoundForTime(now1));
+
+    } finally {
+      transactionPruningPlugin.destroy();
+    }
+  }
+
+  @Test
+  public void testPruneEmptyTable() throws Exception {
+    // Make sure that empty tables do not block the progress of pruning
+
+    // Create an empty table
+    TableName txEmptyTable = TableName.valueOf("emptyPruneTestTable");
+    HTable emptyHTable = createTable(txEmptyTable.getName(), new byte[][]{family}, false,
+                                     Collections.singletonList(TestTransactionProcessor.class.getName()));
+
+    TransactionPruningPlugin transactionPruningPlugin = new TestTransactionPruningPlugin();
+    transactionPruningPlugin.initialize(conf);
+
+    try {
+      long now1 = System.currentTimeMillis();
+      long inactiveTxTimeNow1 = (now1 - 150) * TxConstants.MAX_TX_PER_MS;
+      long noPruneUpperBound = -1;
+      long expectedPruneUpperBound1 = (now1 - 200) * TxConstants.MAX_TX_PER_MS;
+      InMemoryTransactionStateCache.setTransactionSnapshot(
+        new TransactionSnapshot(expectedPruneUpperBound1, expectedPruneUpperBound1, expectedPruneUpperBound1,
+                                ImmutableSet.of(expectedPruneUpperBound1),
+                                ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
+      testUtil.compact(txEmptyTable, true);
+      testUtil.compact(txDataTable1, true);
+      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+      TimeUnit.SECONDS.sleep(2);
+
+      // fetch prune upper bound, there should be no prune upper bound since txEmptyTable cannot be compacted
+      long pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
+      Assert.assertEquals(noPruneUpperBound, pruneUpperBound1);
+      transactionPruningPlugin.pruneComplete(now1, noPruneUpperBound);
+
+      // Now flush the empty table, this will record the table region as empty, and then pruning will continue
+      hBaseAdmin.flush(txEmptyTable);
+      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+      TimeUnit.SECONDS.sleep(2);
+
+      // fetch prune upper bound, again, this time it should work
+      pruneUpperBound1 = transactionPruningPlugin.fetchPruneUpperBound(now1, inactiveTxTimeNow1);
+      Assert.assertEquals(expectedPruneUpperBound1, pruneUpperBound1);
+      transactionPruningPlugin.pruneComplete(now1, expectedPruneUpperBound1);
+
+      // Now add some data to the empty table
+      // (adding data non-transactionally is okay too, we just need some data for the compaction to run)
+      emptyHTable.put(new Put(Bytes.toBytes(1)).addColumn(family, qualifier, Bytes.toBytes(1)));
+      emptyHTable.close();
+
+      // Now run another compaction on txDataTable1 with an updated tx snapshot
+      long now2 = System.currentTimeMillis();
+      long inactiveTxTimeNow2 = (now2 - 150) * TxConstants.MAX_TX_PER_MS;
+      long expectedPruneUpperBound2 = (now2 - 200) * TxConstants.MAX_TX_PER_MS;
+      InMemoryTransactionStateCache.setTransactionSnapshot(
+        new TransactionSnapshot(expectedPruneUpperBound2, expectedPruneUpperBound2, expectedPruneUpperBound2,
+                                ImmutableSet.of(expectedPruneUpperBound2),
+                                ImmutableSortedMap.<Long, TransactionManager.InProgressTx>of()));
+      testUtil.flush(txEmptyTable);
+      testUtil.compact(txDataTable1, true);
+      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+      TimeUnit.SECONDS.sleep(2);
+
+      // Running a prune now should still return min(inactiveTxTimeNow1, expectedPruneUpperBound1) since
+      // txEmptyTable is no longer empty. This information is returned since the txEmptyTable was recorded as being
+      // empty in the previous run with inactiveTxTimeNow1
+      long pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
+      Assert.assertEquals(inactiveTxTimeNow1, pruneUpperBound2);
+      transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound1);
+
+      // However, after compacting txEmptyTable we should get the latest upper bound
+      testUtil.flush(txEmptyTable);
+      testUtil.compact(txEmptyTable, true);
+      // Since the write to prune table happens async, we need to sleep a bit before checking the state of the table
+      TimeUnit.SECONDS.sleep(2);
+      pruneUpperBound2 = transactionPruningPlugin.fetchPruneUpperBound(now2, inactiveTxTimeNow2);
+      Assert.assertEquals(expectedPruneUpperBound2, pruneUpperBound2);
+      transactionPruningPlugin.pruneComplete(now2, expectedPruneUpperBound2);
+    } finally {
+      transactionPruningPlugin.destroy();
+      hBaseAdmin.disableTable(txEmptyTable);
+      hBaseAdmin.deleteTable(txEmptyTable);
+    }
+    }
+
+  private byte[] getRegionName(TableName dataTable, byte[] row) throws IOException {
+    HRegionLocation regionLocation =
+      testUtil.getConnection().getRegionLocator(dataTable).getRegionLocation(row);
+    return regionLocation.getRegionInfo().getRegionName();
+  }
+
+  /**
+   * A transaction co-processor that uses in-memory {@link TransactionSnapshot} for testing
+   */
+  @SuppressWarnings("WeakerAccess")
+  public static class TestTransactionProcessor extends TransactionProcessor {
+    private static final AtomicLong lastMajorCompactionTime = new AtomicLong(-1);
+
+    @Override
+    protected CacheSupplier<TransactionStateCache> getTransactionStateCacheSupplier(RegionCoprocessorEnvironment env) {
+      return new CacheSupplier<TransactionStateCache>() {
+        @Override
+        public TransactionStateCache get() {
+          return new InMemoryTransactionStateCache();
+        }
+
+        @Override
+        public void release() {
+          // no-op
+        }
+      };
+    }
+
+    @Override
+    public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store, StoreFile resultFile,
+                            CompactionRequest request) throws IOException {
+      super.postCompact(e, store, resultFile, request);
+      lastMajorCompactionTime.set(System.currentTimeMillis());
+    }
+  }
+
+  /**
+   * Used to supply in-memory {@link TransactionSnapshot} to {@link TestTransactionProcessor} for testing
+   */
+  @SuppressWarnings("WeakerAccess")
+  public static class InMemoryTransactionStateCache extends TransactionStateCache {
+    private static TransactionVisibilityState transactionSnapshot;
+
+    public static void setTransactionSnapshot(TransactionVisibilityState transactionSnapshot) {
+      InMemoryTransactionStateCache.transactionSnapshot = transactionSnapshot;
+    }
+
+    @Override
+    protected void startUp() throws Exception {
+      // Nothing to do
+    }
+
+    @Override
+    protected void shutDown() throws Exception {
+      // Nothing to do
+    }
+
+    @Override
+    public TransactionVisibilityState getLatestState() {
+      return transactionSnapshot;
+    }
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public static class TestTransactionPruningPlugin extends HBaseTransactionPruningPlugin {
+    @Override
+    protected boolean isTransactionalTable(HTableDescriptor tableDescriptor) {
+      return tableDescriptor.hasCoprocessor(TestTransactionProcessor.class.getName());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplierTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplierTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplierTest.java
new file mode 100644
index 0000000..08d3b49
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplierTest.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Tests to verify the behavior of {@link PruneUpperBoundWriterSupplier}.
+ */
+public class PruneUpperBoundWriterSupplierTest {
+  private static final Logger LOG = LoggerFactory.getLogger(PruneUpperBoundWriterSupplierTest.class);
+  private static final int NUM_OPS = 10000;
+  private static final int NUM_THREADS = 50;
+
+  @Test
+  public void testSupplier() throws Exception {
+    final PruneUpperBoundWriterSupplier supplier = new PruneUpperBoundWriterSupplier(null, null, 10L);
+    // Get one instance now, for later comparisons
+    final PruneUpperBoundWriter writer = supplier.get();
+    final AtomicInteger numOps = new AtomicInteger(NUM_OPS);
+    final Random random = new Random(System.currentTimeMillis());
+
+    // Start threads that will 'get' PruneUpperBoundWriters
+    ExecutorService executor = Executors.newFixedThreadPool(NUM_THREADS);
+    List<Future> futureList = new ArrayList<>();
+    for (int i = 0; i < NUM_THREADS; i++) {
+      futureList.add(executor.submit(new Runnable() {
+
+        @Override
+        public void run() {
+          // Perform NUM_OPS 'gets' of PruneUpperBoundWriter
+          while (numOps.decrementAndGet() > 0) {
+            PruneUpperBoundWriter newWriter = supplier.get();
+            Assert.assertTrue(newWriter == writer);
+            int waitTime = random.nextInt(10);
+            try {
+              TimeUnit.MICROSECONDS.sleep(waitTime);
+            } catch (InterruptedException e) {
+              LOG.warn("Received an exception.", e);
+            }
+          }
+        }
+      }));
+    }
+
+    for (Future future : futureList) {
+      future.get(5, TimeUnit.SECONDS);
+    }
+    executor.shutdown();
+    executor.awaitTermination(2, TimeUnit.SECONDS);
+
+    futureList.clear();
+    numOps.set(NUM_OPS);
+    // Start thread that release PruneUpperBoundWriters
+    executor = Executors.newFixedThreadPool(NUM_THREADS);
+    for (int i = 0; i < NUM_THREADS; i++) {
+      futureList.add(executor.submit(new Runnable() {
+
+        @Override
+        public void run() {
+          // We need to release all NUM_OPS 'gets' that were executed to trigger shutdown of the single instance of
+          // PruneUpperBoundWriter
+          while (numOps.decrementAndGet() > 0) {
+            supplier.release();
+            try {
+              TimeUnit.MICROSECONDS.sleep(random.nextInt(10));
+            } catch (InterruptedException e) {
+              LOG.warn("Received an exception.", e);
+            }
+          }
+        }
+      }));
+    }
+
+    for (Future future : futureList) {
+      future.get(1, TimeUnit.SECONDS);
+    }
+
+    executor.shutdown();
+    executor.awaitTermination(2, TimeUnit.SECONDS);
+
+    // Verify that the PruneUpperBoundWriter is still running and the pruneThread is still alive.
+    Assert.assertTrue(writer.isRunning());
+    Assert.assertTrue(writer.isAlive());
+
+    // Since we got one instance in the beginning, we need to release it
+    supplier.release();
+
+    // Verify that the PruneUpperBoundWriter is shutdown and the pruneThread is not alive anymore.
+    Assert.assertFalse(writer.isRunning());
+    Assert.assertFalse(writer.isAlive());
+  }
+}


[2/5] incubator-tephra git commit: Support for HBase 1.3.x

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
new file mode 100644
index 0000000..558adaa
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * HBase 1.3 specific test for testing {@link CellSkipFilter}.
+ */
+public class CellSkipFilterTest {
+
+  private static final String ROW1KEY = "row1";
+  private static final String ROW2KEY = "row2";
+  private static final String FAM1KEY = "fam1";
+  private static final String COL1KEY = "col1";
+  private static final String FAM2KEY = "fam2";
+  private static final String COL2KEY = "col2";
+  private static final String VALUE = "value";
+
+  @Test
+  public void testSkipFiltering() throws Exception {
+    long timestamp = System.currentTimeMillis();
+    // Test to check that we get NEXT_COL once the INCLUDE_AND_NEXT_COL is returned for the same key
+    Filter filter = new CellSkipFilter(new MyFilter(0));
+    assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                              timestamp)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY,
+                                                                                           VALUE, timestamp - 1)));
+
+    // Next call should get NEXT_COL instead of SKIP, as it would be returned by CellSkipFilter
+    assertEquals(Filter.ReturnCode.NEXT_COL, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                               timestamp - 2)));
+
+    // Next call with the same key should return the NEXT_COL again, as it would be returned by CellSkipFilter
+    assertEquals(Filter.ReturnCode.NEXT_COL, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                               timestamp - 3)));
+
+    // Since MyFilter counter is not incremented in the previous call, filtering for the different keyvalue should
+    // give SKIP from MyFilter
+    assertEquals(Filter.ReturnCode.SKIP, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM2KEY, COL1KEY, VALUE,
+                                                                           timestamp - 4)));
+
+    // Test to check that we get NEXT_COL once the NEXT_COL is returned for the same key
+    filter = new CellSkipFilter(new MyFilter(2));
+    assertEquals(Filter.ReturnCode.SKIP, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                           timestamp)));
+    assertEquals(Filter.ReturnCode.NEXT_COL, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                               timestamp - 1)));
+
+    // Next call should get NEXT_COL instead of NEXT_ROW, as it would be returned by CellSkipFilter
+    assertEquals(Filter.ReturnCode.NEXT_COL, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                               timestamp - 2)));
+
+    // Next call with the same key should return the NEXT_COL again, as it would be returned by CellSkipFilter
+    assertEquals(Filter.ReturnCode.NEXT_COL, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL1KEY, VALUE,
+                                                                               timestamp - 3)));
+
+    // Since MyFilter counter is not incremented in the previous call, filtering for the different keyvalue should
+    // give NEXT_ROW from MyFilter
+    assertEquals(Filter.ReturnCode.NEXT_ROW, filter.filterKeyValue(newKeyValue(ROW1KEY, FAM1KEY, COL2KEY, VALUE,
+                                                                               timestamp - 4)));
+
+    // Next call with the new key should returned the SEEK_NEXT_USING_HINT
+    assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(newKeyValue(ROW2KEY, FAM1KEY, COL1KEY,
+                                                                                           VALUE, timestamp - 5)));
+  }
+
+  private KeyValue newKeyValue(String rowkey, String family, String column, String value, long timestamp) {
+    return new KeyValue(Bytes.toBytes(rowkey), Bytes.toBytes(family), Bytes.toBytes(column),
+                        timestamp, Bytes.toBytes(value));
+  }
+
+  /**
+   * Sample filter for testing. This filter maintains the {@link List} of {@link ReturnCode}s. It accepts the
+   * start index in the list and start serving the return codes corresponding that that index. Every time the
+   * return code is served, index is incremented.
+   */
+  class MyFilter extends FilterBase {
+
+    private final List<ReturnCode> returnCodes;
+    private int counter;
+
+    public MyFilter(int startIndex) {
+      returnCodes = Arrays.asList(ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.SKIP,
+                                  ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW, ReturnCode.SEEK_NEXT_USING_HINT);
+      counter = startIndex;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(Cell cell) throws IOException {
+      ReturnCode code = returnCodes.get(counter % returnCodes.size());
+      counter++;
+      return code;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
new file mode 100644
index 0000000..15842a3
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
@@ -0,0 +1,624 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MockRegionServerServices;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.tephra.ChangeId;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.TransactionManager;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.coprocessor.TransactionStateCache;
+import org.apache.tephra.coprocessor.TransactionStateCacheSupplier;
+import org.apache.tephra.manager.InvalidTxList;
+import org.apache.tephra.metrics.TxMetricsCollector;
+import org.apache.tephra.persist.HDFSTransactionStateStorage;
+import org.apache.tephra.persist.TransactionSnapshot;
+import org.apache.tephra.persist.TransactionVisibilityState;
+import org.apache.tephra.snapshot.DefaultSnapshotCodec;
+import org.apache.tephra.snapshot.SnapshotCodecProvider;
+import org.apache.tephra.util.TxUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests filtering of invalid transaction data by the {@link TransactionProcessor} coprocessor.
+ */
+public class TransactionProcessorTest {
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionProcessorTest.class);
+
+  // 8 versions, 1 hour apart, latest is current ts.
+  private static final long[] V;
+
+  static {
+    long now = System.currentTimeMillis();
+    V = new long[9];
+    for (int i = 0; i < V.length; i++) {
+      V[i] = (now - TimeUnit.HOURS.toMillis(8 - i)) * TxConstants.MAX_TX_PER_MS;
+    }
+  }
+
+  @ClassRule
+  public static TemporaryFolder tmpFolder = new TemporaryFolder();
+  private static MiniDFSCluster dfsCluster;
+  private static Configuration conf;
+  private static LongArrayList invalidSet = new LongArrayList(new long[]{V[3], V[5], V[7]});
+  private static TransactionVisibilityState txVisibilityState;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    Configuration hConf = new Configuration();
+    String rootDir = tmpFolder.newFolder().getAbsolutePath();
+    hConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, rootDir);
+    hConf.set(HConstants.HBASE_DIR, rootDir + "/hbase");
+
+    dfsCluster = new MiniDFSCluster.Builder(hConf).numDataNodes(1).build();
+    dfsCluster.waitActive();
+    conf = HBaseConfiguration.create(dfsCluster.getFileSystem().getConf());
+
+    conf.unset(TxConstants.Manager.CFG_TX_HDFS_USER);
+    conf.unset(TxConstants.Persist.CFG_TX_SNAPHOT_CODEC_CLASSES);
+    String localTestDir = tmpFolder.newFolder().getAbsolutePath();
+    conf.set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, localTestDir);
+    conf.set(TxConstants.Persist.CFG_TX_SNAPHOT_CODEC_CLASSES, DefaultSnapshotCodec.class.getName());
+
+    // write an initial transaction snapshot
+    InvalidTxList invalidTxList = new InvalidTxList();
+    invalidTxList.addAll(invalidSet);
+    TransactionSnapshot txSnapshot = TransactionSnapshot.copyFrom(
+        System.currentTimeMillis(), V[6] - 1, V[7], invalidTxList,
+        // this will set visibility upper bound to V[6]
+        Maps.newTreeMap(ImmutableSortedMap.of(V[6], new TransactionManager.InProgressTx(
+          V[6] - 1, Long.MAX_VALUE, TransactionManager.InProgressType.SHORT))),
+        new HashMap<Long, Set<ChangeId>>(), new TreeMap<Long, Set<ChangeId>>());
+    txVisibilityState = new TransactionSnapshot(txSnapshot.getTimestamp(), txSnapshot.getReadPointer(),
+                                                txSnapshot.getWritePointer(), txSnapshot.getInvalid(),
+                                                txSnapshot.getInProgress());
+    HDFSTransactionStateStorage tmpStorage =
+      new HDFSTransactionStateStorage(conf, new SnapshotCodecProvider(conf), new TxMetricsCollector());
+    tmpStorage.startAndWait();
+    tmpStorage.writeSnapshot(txSnapshot);
+    tmpStorage.stopAndWait();
+  }
+
+  @AfterClass
+  public static void shutdownAfterClass() throws Exception {
+    dfsCluster.shutdown();
+  }
+
+  @Test
+  public void testDataJanitorRegionScanner() throws Exception {
+    String tableName = "TestRegionScanner";
+    byte[] familyBytes = Bytes.toBytes("f");
+    byte[] columnBytes = Bytes.toBytes("c");
+    HRegion region = createRegion(tableName, familyBytes, TimeUnit.HOURS.toMillis(3));
+    try {
+      region.initialize();
+      TransactionStateCache cache = new TransactionStateCacheSupplier(conf).get();
+      LOG.info("Coprocessor is using transaction state: " + cache.getLatestState());
+
+      for (int i = 1; i <= 8; i++) {
+        for (int k = 1; k <= i; k++) {
+          Put p = new Put(Bytes.toBytes(i));
+          p.add(familyBytes, columnBytes, V[k], Bytes.toBytes(V[k]));
+          region.put(p);
+        }
+      }
+
+      List<Cell> results = Lists.newArrayList();
+
+      // force a flush to clear the data
+      // during flush, the coprocessor should drop all KeyValues with timestamps in the invalid set
+
+      LOG.info("Flushing region " + region.getRegionInfo().getRegionNameAsString());
+      region.flushcache(true, false);
+
+      // now a normal scan should only return the valid rows
+      // do not use a filter here to test that cleanup works on flush
+      Scan scan = new Scan();
+      scan.setMaxVersions(10);
+      RegionScanner regionScanner = region.getScanner(scan);
+
+      // first returned value should be "4" with version "4"
+      results.clear();
+      assertTrue(regionScanner.next(results));
+      assertKeyValueMatches(results, 4, new long[]{V[4]});
+
+      results.clear();
+      assertTrue(regionScanner.next(results));
+      assertKeyValueMatches(results, 5, new long[] {V[4]});
+
+      results.clear();
+      assertTrue(regionScanner.next(results));
+      assertKeyValueMatches(results, 6, new long[]{V[6], V[4]});
+
+      results.clear();
+      assertTrue(regionScanner.next(results));
+      assertKeyValueMatches(results, 7, new long[]{V[6], V[4]});
+
+      results.clear();
+      assertFalse(regionScanner.next(results));
+      assertKeyValueMatches(results, 8, new long[] {V[8], V[6], V[4]});
+    } finally {
+      region.close();
+    }
+  }
+
+  @Test
+  public void testDeleteFiltering() throws Exception {
+    String tableName = "TestDeleteFiltering";
+    byte[] familyBytes = Bytes.toBytes("f");
+    byte[] columnBytes = Bytes.toBytes("c");
+    HRegion region = createRegion(tableName, familyBytes, 0);
+    try {
+      region.initialize();
+      TransactionStateCache cache = new TransactionStateCacheSupplier(conf).get();
+      LOG.info("Coprocessor is using transaction state: " + cache.getLatestState());
+
+      byte[] row = Bytes.toBytes(1);
+      for (int i = 4; i < V.length; i++) {
+        Put p = new Put(row);
+        p.add(familyBytes, columnBytes, V[i], Bytes.toBytes(V[i]));
+        region.put(p);
+      }
+
+      // delete from the third entry back
+      // take that cell's timestamp + 1 to simulate a delete in a new tx
+      long deleteTs = V[5] + 1;
+      Delete d = new Delete(row, deleteTs);
+      LOG.info("Issuing delete at timestamp " + deleteTs);
+      // row deletes are not yet supported (TransactionAwareHTable normally handles this)
+      d.deleteColumns(familyBytes, columnBytes);
+      region.delete(d);
+
+      List<Cell> results = Lists.newArrayList();
+
+      // force a flush to clear the data
+      // during flush, we should drop the deleted version, but not the others
+      LOG.info("Flushing region " + region.getRegionInfo().getRegionNameAsString());
+      region.flushcache(true, false);
+
+      // now a normal scan should return row with versions at: V[8], V[6].
+      // V[7] is invalid and V[5] and prior are deleted.
+      Scan scan = new Scan();
+      scan.setMaxVersions(10);
+      RegionScanner regionScanner = region.getScanner(scan);
+      // should be only one row
+      assertFalse(regionScanner.next(results));
+      assertKeyValueMatches(results, 1,
+          new long[]{V[8], V[6], deleteTs},
+          new byte[][]{Bytes.toBytes(V[8]), Bytes.toBytes(V[6]), new byte[0]});
+    } finally {
+      region.close();
+    }
+  }
+
+  @Test
+  public void testDeleteMarkerCleanup() throws Exception {
+    String tableName = "TestDeleteMarkerCleanup";
+    byte[] familyBytes = Bytes.toBytes("f");
+    HRegion region = createRegion(tableName, familyBytes, 0);
+    try {
+      region.initialize();
+
+      // all puts use a timestamp before the tx snapshot's visibility upper bound, making them eligible for removal
+      long writeTs = txVisibilityState.getVisibilityUpperBound() - 10;
+      // deletes are performed after the writes, but still before the visibility upper bound
+      long deleteTs = writeTs + 1;
+      // write separate columns to confirm that delete markers survive across flushes
+      byte[] row = Bytes.toBytes(100);
+      Put p = new Put(row);
+
+      LOG.info("Writing columns at timestamp " + writeTs);
+      for (int i = 0; i < 5; i++) {
+        byte[] iBytes = Bytes.toBytes(i);
+        p.add(familyBytes, iBytes, writeTs, iBytes);
+      }
+      region.put(p);
+      // read all back
+      Scan scan = new Scan(row);
+      RegionScanner regionScanner = region.getScanner(scan);
+      List<Cell> results = Lists.newArrayList();
+      assertFalse(regionScanner.next(results));
+      for (int i = 0; i < 5; i++) {
+        Cell cell = results.get(i);
+        assertArrayEquals(row, cell.getRow());
+        byte[] idxBytes = Bytes.toBytes(i);
+        assertArrayEquals(idxBytes, cell.getQualifier());
+        assertArrayEquals(idxBytes, cell.getValue());
+      }
+
+      // force a flush to clear the memstore
+      LOG.info("Before delete, flushing region " + region.getRegionInfo().getRegionNameAsString());
+      region.flushcache(false, false);
+
+      // delete the odd entries
+      for (int i = 0; i < 5; i++) {
+        if (i % 2 == 1) {
+          // deletes are performed as puts with empty values
+          Put deletePut = new Put(row);
+          deletePut.add(familyBytes, Bytes.toBytes(i), deleteTs, new byte[0]);
+          region.put(deletePut);
+        }
+      }
+
+      // read all back
+      scan = new Scan(row);
+      scan.setFilter(TransactionFilters.getVisibilityFilter(TxUtils.createDummyTransaction(txVisibilityState),
+                                                            new TreeMap<byte[], Long>(), false, ScanType.USER_SCAN));
+      regionScanner = region.getScanner(scan);
+      results = Lists.newArrayList();
+      assertFalse(regionScanner.next(results));
+      assertEquals(3, results.size());
+      // only even columns should exist
+      for (int i = 0; i < 3; i++) {
+        Cell cell = results.get(i);
+        LOG.info("Got cell " + cell);
+        assertArrayEquals(row, cell.getRow());
+        byte[] idxBytes = Bytes.toBytes(i * 2);
+        assertArrayEquals(idxBytes, cell.getQualifier());
+        assertArrayEquals(idxBytes, cell.getValue());
+      }
+
+      // force another flush on the delete markers
+      // during flush, we should retain the delete markers, since they can only safely be dropped by a major compaction
+      LOG.info("After delete, flushing region " + region.getRegionInfo().getRegionNameAsString());
+      region.flushcache(true, false);
+
+      scan = new Scan(row);
+      scan.setFilter(TransactionFilters.getVisibilityFilter(TxUtils.createDummyTransaction(txVisibilityState),
+                                                            new TreeMap<byte[], Long>(), false, ScanType.USER_SCAN));
+
+      regionScanner = region.getScanner(scan);
+      results = Lists.newArrayList();
+      assertFalse(regionScanner.next(results));
+      assertEquals(3, results.size());
+      // only even columns should exist
+      for (int i = 0; i < 3; i++) {
+        Cell cell = results.get(i);
+        assertArrayEquals(row, cell.getRow());
+        byte[] idxBytes = Bytes.toBytes(i * 2);
+        assertArrayEquals(idxBytes, cell.getQualifier());
+        assertArrayEquals(idxBytes, cell.getValue());
+      }
+
+      // force a major compaction
+      LOG.info("Forcing major compaction of region " + region.getRegionInfo().getRegionNameAsString());
+      region.compact(true);
+
+      // perform a raw scan (no filter) to confirm that the delete markers are now gone
+      scan = new Scan(row);
+      regionScanner = region.getScanner(scan);
+      results = Lists.newArrayList();
+      assertFalse(regionScanner.next(results));
+      assertEquals(3, results.size());
+      // only even columns should exist
+      for (int i = 0; i < 3; i++) {
+        Cell cell = results.get(i);
+        assertArrayEquals(row, cell.getRow());
+        byte[] idxBytes = Bytes.toBytes(i * 2);
+        assertArrayEquals(idxBytes, cell.getQualifier());
+        assertArrayEquals(idxBytes, cell.getValue());
+      }
+    } finally {
+      region.close();
+    }
+  }
+
+  /**
+   * Test that we correctly preserve the timestamp set for column family delete markers.  This is not
+   * directly required for the TransactionAwareHTable usage, but is the right thing to do and ensures
+   * that we make it easy to interoperate with other systems.
+   */
+  @Test
+  public void testFamilyDeleteTimestamp() throws Exception {
+    String tableName = "TestFamilyDeleteTimestamp";
+    byte[] family1Bytes = Bytes.toBytes("f1");
+    byte[] columnBytes = Bytes.toBytes("c");
+    byte[] rowBytes = Bytes.toBytes("row");
+    byte[] valBytes = Bytes.toBytes("val");
+    HRegion region = createRegion(tableName, family1Bytes, 0);
+    try {
+      region.initialize();
+
+      long now = System.currentTimeMillis() * TxConstants.MAX_TX_PER_MS;
+      Put p = new Put(rowBytes);
+      p.add(family1Bytes, columnBytes, now - 10, valBytes);
+      region.put(p);
+
+      // issue a family delete with an explicit timestamp
+      Delete delete = new Delete(rowBytes, now);
+      delete.deleteFamily(family1Bytes, now - 5);
+      region.delete(delete);
+
+      // test that the delete marker preserved the timestamp
+      Scan scan = new Scan();
+      scan.setMaxVersions();
+      RegionScanner scanner = region.getScanner(scan);
+      List<Cell> results = Lists.newArrayList();
+      scanner.next(results);
+      assertEquals(2, results.size());
+      // delete marker should appear first
+      Cell cell = results.get(0);
+      assertArrayEquals(new byte[0], cell.getQualifier());
+      assertArrayEquals(new byte[0], cell.getValue());
+      assertEquals(now - 5, cell.getTimestamp());
+      // since this is an unfiltered scan against the region, the original put should be next
+      cell = results.get(1);
+      assertArrayEquals(valBytes, cell.getValue());
+      assertEquals(now - 10, cell.getTimestamp());
+      scanner.close();
+
+
+      // with a filtered scan the original put should disappear
+      scan = new Scan();
+      scan.setMaxVersions();
+      scan.setFilter(TransactionFilters.getVisibilityFilter(TxUtils.createDummyTransaction(txVisibilityState),
+                                                            new TreeMap<byte[], Long>(), false, ScanType.USER_SCAN));
+      scanner = region.getScanner(scan);
+      results = Lists.newArrayList();
+      scanner.next(results);
+      assertEquals(0, results.size());
+      scanner.close();
+    } finally {
+      region.close();
+    }
+  }
+
+  @Test
+  public void testPreExistingData() throws Exception {
+    String tableName = "TestPreExistingData";
+    byte[] familyBytes = Bytes.toBytes("f");
+    long ttlMillis = TimeUnit.DAYS.toMillis(14);
+    HRegion region = createRegion(tableName, familyBytes, ttlMillis);
+    try {
+      region.initialize();
+
+      // timestamps for pre-existing, non-transactional data
+      long now = txVisibilityState.getVisibilityUpperBound() / TxConstants.MAX_TX_PER_MS;
+      long older = now - ttlMillis / 2;
+      long newer = now - ttlMillis / 3;
+      // timestamps for transactional data
+      long nowTx = txVisibilityState.getVisibilityUpperBound();
+      long olderTx = nowTx - (ttlMillis / 2) * TxConstants.MAX_TX_PER_MS;
+      long newerTx = nowTx - (ttlMillis / 3) * TxConstants.MAX_TX_PER_MS;
+
+      Map<byte[], Long> ttls = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+      ttls.put(familyBytes, ttlMillis);
+
+      List<Cell> cells = new ArrayList<>();
+      cells.add(new KeyValue(Bytes.toBytes("r1"), familyBytes, Bytes.toBytes("c1"), older, Bytes.toBytes("v11")));
+      cells.add(new KeyValue(Bytes.toBytes("r1"), familyBytes, Bytes.toBytes("c2"), newer, Bytes.toBytes("v12")));
+      cells.add(new KeyValue(Bytes.toBytes("r2"), familyBytes, Bytes.toBytes("c1"), older, Bytes.toBytes("v21")));
+      cells.add(new KeyValue(Bytes.toBytes("r2"), familyBytes, Bytes.toBytes("c2"), newer, Bytes.toBytes("v22")));
+      cells.add(new KeyValue(Bytes.toBytes("r3"), familyBytes, Bytes.toBytes("c1"), olderTx, Bytes.toBytes("v31")));
+      cells.add(new KeyValue(Bytes.toBytes("r3"), familyBytes, Bytes.toBytes("c2"), newerTx, Bytes.toBytes("v32")));
+
+      // Write non-transactional and transactional data
+      for (Cell c : cells) {
+        region.put(new Put(c.getRow()).add(c.getFamily(), c.getQualifier(), c.getTimestamp(), c.getValue()));
+      }
+
+      Scan rawScan = new Scan();
+      rawScan.setMaxVersions();
+
+      Transaction dummyTransaction = TxUtils.createDummyTransaction(txVisibilityState);
+      Scan txScan = new Scan();
+      txScan.setMaxVersions();
+      txScan.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttls, dummyTransaction, true),
+                          TxUtils.getMaxVisibleTimestamp(dummyTransaction));
+      txScan.setFilter(TransactionFilters.getVisibilityFilter(dummyTransaction, ttls, false, ScanType.USER_SCAN));
+
+      // read all back with raw scanner
+      scanAndAssert(region, cells, rawScan);
+
+      // read all back with transaction filter
+      scanAndAssert(region, cells, txScan);
+
+      // force a flush to clear the memstore
+      region.flushcache(true, false);
+      scanAndAssert(region, cells, txScan);
+
+      // force a major compaction to remove any expired cells
+      region.compact(true);
+      scanAndAssert(region, cells, txScan);
+
+      // Reduce TTL, this should make cells with timestamps older and olderTx expire
+      long newTtl = ttlMillis / 2 - 1;
+      region = updateTtl(region, familyBytes, newTtl);
+      ttls.put(familyBytes, newTtl);
+      txScan.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttls, dummyTransaction, true),
+                          TxUtils.getMaxVisibleTimestamp(dummyTransaction));
+      txScan.setFilter(TransactionFilters.getVisibilityFilter(dummyTransaction, ttls, false, ScanType.USER_SCAN));
+
+      // Raw scan should still give all cells
+      scanAndAssert(region, cells, rawScan);
+      // However, tx scan should not return expired cells
+      scanAndAssert(region, select(cells, 1, 3, 5), txScan);
+
+      region.flushcache(true, false);
+      scanAndAssert(region, cells, rawScan);
+
+      // force a major compaction to remove any expired cells
+      region.compact(true);
+      // This time raw scan too should not return expired cells, as they would be dropped during major compaction
+      scanAndAssert(region, select(cells, 1, 3, 5), rawScan);
+
+      // Reduce TTL again to 1 ms, this should expire all cells
+      newTtl = 1;
+      region = updateTtl(region, familyBytes, newTtl);
+      ttls.put(familyBytes, newTtl);
+      txScan.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttls, dummyTransaction, true),
+                          TxUtils.getMaxVisibleTimestamp(dummyTransaction));
+      txScan.setFilter(TransactionFilters.getVisibilityFilter(dummyTransaction, ttls, false, ScanType.USER_SCAN));
+
+      // force a major compaction to remove expired cells
+      region.compact(true);
+      // This time raw scan should not return any cells, as all cells have expired.
+      scanAndAssert(region, Collections.<Cell>emptyList(), rawScan);
+    } finally {
+      region.close();
+    }
+  }
+
+  private List<Cell> select(List<Cell> cells, int... indexes) {
+    List<Cell> newCells = new ArrayList<>();
+    for (int i : indexes) {
+      newCells.add(cells.get(i));
+    }
+    return newCells;
+  }
+
+  @SuppressWarnings("StatementWithEmptyBody")
+  private void scanAndAssert(HRegion region, List<Cell> expected, Scan scan) throws Exception {
+    try (RegionScanner regionScanner = region.getScanner(scan)) {
+      List<Cell> results = Lists.newArrayList();
+      while (regionScanner.next(results)) { }
+      assertEquals(expected, results);
+    }
+  }
+
+  private HRegion updateTtl(HRegion region, byte[] family, long ttl) throws Exception {
+    region.close();
+    HTableDescriptor htd = region.getTableDesc();
+    HColumnDescriptor cfd = htd.getFamily(family);
+    if (ttl > 0) {
+      cfd.setValue(TxConstants.PROPERTY_TTL, String.valueOf(ttl));
+    }
+    cfd.setMaxVersions(10);
+    return HRegion.openHRegion(region.getRegionInfo(), htd, region.getWAL(), conf,
+                               new LocalRegionServerServices(conf, ServerName.valueOf(
+                                 InetAddress.getLocalHost().getHostName(), 0, System.currentTimeMillis())), null);
+  }
+
+  private HRegion createRegion(String tableName, byte[] family, long ttl) throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
+    HColumnDescriptor cfd = new HColumnDescriptor(family);
+    if (ttl > 0) {
+      cfd.setValue(TxConstants.PROPERTY_TTL, String.valueOf(ttl));
+    }
+    cfd.setMaxVersions(10);
+    htd.addFamily(cfd);
+    htd.addCoprocessor(TransactionProcessor.class.getName());
+    Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), htd.getTableName());
+    FileSystem fs = FileSystem.get(conf);
+    assertTrue(fs.mkdirs(tablePath));
+    WALFactory walFactory = new WALFactory(conf, null, tableName + ".hlog");
+    WAL hLog = walFactory.getWAL(new byte[]{1}, null);
+    HRegionInfo regionInfo = new HRegionInfo(TableName.valueOf(tableName));
+    HRegionFileSystem regionFS = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tablePath, regionInfo);
+    return new HRegion(regionFS, hLog, conf, htd,
+        new LocalRegionServerServices(conf, ServerName.valueOf(
+            InetAddress.getLocalHost().getHostName(), 0, System.currentTimeMillis())));
+  }
+
+  private void assertKeyValueMatches(List<Cell> results, int index, long[] versions) {
+    byte[][] values = new byte[versions.length][];
+    for (int i = 0; i < versions.length; i++) {
+      values[i] = Bytes.toBytes(versions[i]);
+    }
+    assertKeyValueMatches(results, index, versions, values);
+  }
+
+  private void assertKeyValueMatches(List<Cell> results, int index, long[] versions, byte[][] values) {
+    assertEquals(versions.length, results.size());
+    assertEquals(values.length, results.size());
+    for (int i = 0; i < versions.length; i++) {
+      Cell kv = results.get(i);
+      assertArrayEquals(Bytes.toBytes(index), kv.getRow());
+      assertEquals(versions[i], kv.getTimestamp());
+      assertArrayEquals(values[i], kv.getValue());
+    }
+  }
+
+  @Test
+  public void testTransactionStateCache() throws Exception {
+    TransactionStateCache cache = new TransactionStateCache();
+    cache.setConf(conf);
+    cache.startAndWait();
+    // verify that the transaction snapshot read matches what we wrote in setupBeforeClass()
+    TransactionVisibilityState cachedSnapshot = cache.getLatestState();
+    assertNotNull(cachedSnapshot);
+    assertEquals(invalidSet, cachedSnapshot.getInvalid());
+    cache.stopAndWait();
+  }
+
+  private static class LocalRegionServerServices extends MockRegionServerServices {
+    private final ServerName serverName;
+
+    public LocalRegionServerServices(Configuration conf, ServerName serverName) {
+      super(conf);
+      this.serverName = serverName;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return serverName;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
new file mode 100644
index 0000000..d976085
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
@@ -0,0 +1,374 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.hbase.AbstractTransactionVisibilityFilterTest;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * HBase 1.1 specific test for filtering logic applied when reading data transactionally.
+ */
+public class TransactionVisibilityFilterTest extends AbstractTransactionVisibilityFilterTest {
+  /**
+   * Test filtering of KeyValues for in-progress and invalid transactions.
+   * @throws Exception
+   */
+  @Test
+  public void testFiltering() throws Exception {
+    TxFilterFactory txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new TransactionVisibilityFilter(tx, familyTTLs, false, ScanType.USER_SCAN);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL));
+  }
+
+  @Test
+  public void testSubFilter() throws Exception {
+    final FilterBase includeFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.INCLUDE;
+      }
+    };
+    TxFilterFactory txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new TransactionVisibilityFilter(tx, familyTTLs, false, ScanType.USER_SCAN, includeFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL));
+
+    final Filter skipFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.SKIP;
+      }
+    };
+    txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new TransactionVisibilityFilter(tx, familyTTLs, false, ScanType.USER_SCAN, skipFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL));
+
+    final Filter includeNextFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+    };
+    txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new TransactionVisibilityFilter(tx, familyTTLs, false, ScanType.USER_SCAN, includeNextFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL));
+
+    final Filter nextColFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.NEXT_COL;
+      }
+    };
+    txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new TransactionVisibilityFilter(tx, familyTTLs, false, ScanType.USER_SCAN, nextColFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL));
+
+  }
+
+  @Test
+  public void testSubFilterOverride() throws Exception {
+    final FilterBase includeFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.INCLUDE;
+      }
+    };
+    TxFilterFactory txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new CustomTxFilter(tx, familyTTLs, false, ScanType.USER_SCAN, includeFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.INCLUDE,
+                                      Filter.ReturnCode.INCLUDE,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.INCLUDE,
+                                      Filter.ReturnCode.INCLUDE));
+
+    final Filter skipFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.SKIP;
+      }
+    };
+    txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new CustomTxFilter(tx, familyTTLs, false, ScanType.USER_SCAN, skipFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL));
+
+    final Filter includeNextFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+    };
+    txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new CustomTxFilter(tx, familyTTLs, false, ScanType.USER_SCAN, includeNextFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                                      Filter.ReturnCode.INCLUDE_AND_NEXT_COL));
+
+    final Filter nextColFilter = new FilterBase() {
+      @Override
+      public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+        return ReturnCode.NEXT_COL;
+      }
+    };
+    txFilterFactory = new TxFilterFactory() {
+      @Override
+      public Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs) {
+        return new CustomTxFilter(tx, familyTTLs, false, ScanType.USER_SCAN, nextColFilter);
+      }
+    };
+    runFilteringTest(txFilterFactory,
+                     ImmutableList.of(Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.SKIP,
+                                      Filter.ReturnCode.NEXT_COL,
+                                      Filter.ReturnCode.NEXT_COL));
+
+  }
+
+  private void runFilteringTest(TxFilterFactory txFilterFactory,
+                                List<Filter.ReturnCode> assertCodes) throws Exception {
+
+    /*
+     * Start and stop some transactions.  This will give us a transaction state something like the following
+     * (numbers only reflect ordering, not actual transaction IDs):
+     *   6  - in progress
+     *   5  - committed
+     *   4  - invalid
+     *   3  - in-progress
+     *   2  - committed
+     *   1  - committed
+     *
+     *   read ptr = 5
+     *   write ptr = 6
+     */
+
+    Transaction tx1 = txManager.startShort();
+    assertTrue(txManager.canCommit(tx1, EMPTY_CHANGESET));
+    assertTrue(txManager.commit(tx1));
+
+    Transaction tx2 = txManager.startShort();
+    assertTrue(txManager.canCommit(tx2, EMPTY_CHANGESET));
+    assertTrue(txManager.commit(tx2));
+
+    Transaction tx3 = txManager.startShort();
+    Transaction tx4 = txManager.startShort();
+    txManager.invalidate(tx4.getTransactionId());
+
+    Transaction tx5 = txManager.startShort();
+    assertTrue(txManager.canCommit(tx5, EMPTY_CHANGESET));
+    assertTrue(txManager.commit(tx5));
+
+    Transaction tx6 = txManager.startShort();
+
+    Map<byte[], Long> ttls = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+    Filter filter = txFilterFactory.getTxFilter(tx6, ttls);
+
+    assertEquals(assertCodes.get(5),
+                 filter.filterKeyValue(newKeyValue("row1", "val1", tx6.getTransactionId())));
+    assertEquals(assertCodes.get(4),
+                 filter.filterKeyValue(newKeyValue("row1", "val1", tx5.getTransactionId())));
+    assertEquals(assertCodes.get(3),
+                 filter.filterKeyValue(newKeyValue("row1", "val1", tx4.getTransactionId())));
+    assertEquals(assertCodes.get(2),
+                 filter.filterKeyValue(newKeyValue("row1", "val1", tx3.getTransactionId())));
+    assertEquals(assertCodes.get(1),
+                 filter.filterKeyValue(newKeyValue("row1", "val1", tx2.getTransactionId())));
+    assertEquals(assertCodes.get(0),
+                 filter.filterKeyValue(newKeyValue("row1", "val1", tx1.getTransactionId())));
+  }
+
+  /**
+   * Test filtering for TTL settings.
+   * @throws Exception
+   */
+  @Test
+  public void testTTLFiltering() throws Exception {
+    Map<byte[], Long> ttls = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+    ttls.put(FAM, 10L);
+    ttls.put(FAM2, 30L);
+    ttls.put(FAM3, 0L);
+
+    Transaction tx = txManager.startShort();
+    long now = tx.getVisibilityUpperBound();
+    Filter filter = new TransactionVisibilityFilter(tx, ttls, false, ScanType.USER_SCAN);
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", now)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", now - 1 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", now - 11 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM2, "val1", now - 11 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM2, "val1", now - 21 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM2, "val1", now - 31 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM3, "val1", now - 31 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM3, "val1", now - 1001 * TxConstants.MAX_TX_PER_MS)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row2", FAM, "val1", now)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row2", FAM, "val1", now - 1 * TxConstants.MAX_TX_PER_MS)));
+
+    // Verify ttl for pre-existing, non-transactional data
+    long preNow = now / TxConstants.MAX_TX_PER_MS;
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", preNow)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", preNow - 9L)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", preNow - 10L)));
+    assertEquals(Filter.ReturnCode.NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM, "val1", preNow - 11L)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM3, "val1", preNow)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM3, "val1", preNow - 9L)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM3, "val1", preNow - 10L)));
+    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL,
+                 filter.filterKeyValue(newKeyValue("row1", FAM3, "val1", preNow - 1001L)));
+  }
+
+  protected KeyValue newKeyValue(String rowkey, String value, long timestamp) {
+    return new KeyValue(Bytes.toBytes(rowkey), FAM, COL, timestamp, Bytes.toBytes(value));
+  }
+
+  protected KeyValue newKeyValue(String rowkey, byte[] family, String value, long timestamp) {
+    return new KeyValue(Bytes.toBytes(rowkey), family, COL, timestamp, Bytes.toBytes(value));
+  }
+
+  private interface TxFilterFactory {
+    Filter getTxFilter(Transaction tx, Map<byte[], Long> familyTTLs);
+  }
+
+  private class CustomTxFilter extends TransactionVisibilityFilter {
+    public CustomTxFilter(Transaction tx, Map<byte[], Long> ttlByFamily, boolean allowEmptyValues, ScanType scanType,
+                          @Nullable Filter cellFilter) {
+      super(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter);
+    }
+
+    @Override
+    protected ReturnCode determineReturnCode(ReturnCode txFilterCode, ReturnCode subFilterCode) {
+      switch (subFilterCode) {
+        case INCLUDE:
+          return ReturnCode.INCLUDE;
+        case INCLUDE_AND_NEXT_COL:
+          return ReturnCode.INCLUDE_AND_NEXT_COL;
+        case SKIP:
+          return txFilterCode == ReturnCode.INCLUDE ? ReturnCode.SKIP : ReturnCode.NEXT_COL;
+        default:
+          return subFilterCode;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/DataJanitorStateTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/DataJanitorStateTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/DataJanitorStateTest.java
new file mode 100644
index 0000000..2e9dc17
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/txprune/DataJanitorStateTest.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedSet;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.hbase.AbstractHBaseTableTest;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+/**
+ * Test methods of {@link DataJanitorState}
+ */
+// TODO: Group all the tests that need HBase mini cluster into a suite, so that we start the mini-cluster only once
+public class DataJanitorStateTest extends AbstractHBaseTableTest {
+
+  private TableName pruneStateTable;
+  private DataJanitorState dataJanitorState;
+
+  @Before
+  public void beforeTest() throws Exception {
+    pruneStateTable = TableName.valueOf(conf.get(TxConstants.TransactionPruning.PRUNE_STATE_TABLE,
+                                                 TxConstants.TransactionPruning.DEFAULT_PRUNE_STATE_TABLE));
+    HTable table = createTable(pruneStateTable.getName(), new byte[][]{DataJanitorState.FAMILY}, false,
+                               // Prune state table is a non-transactional table, hence no transaction co-processor
+                               Collections.<String>emptyList());
+    table.close();
+
+    dataJanitorState =
+      new DataJanitorState(new DataJanitorState.TableSupplier() {
+        @Override
+        public Table get() throws IOException {
+          return testUtil.getConnection().getTable(pruneStateTable);
+        }
+      });
+
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    hBaseAdmin.disableTable(pruneStateTable);
+    hBaseAdmin.deleteTable(pruneStateTable);
+  }
+
+  @Test
+  public void testSavePruneUpperBound() throws Exception {
+    int max = 20;
+
+    // Nothing should be present in the beginning
+    Assert.assertEquals(-1, dataJanitorState.getPruneUpperBoundForRegion(Bytes.toBytes(10L)));
+
+    // Save some region - prune upper bound values
+    // We should have values for regions 0, 2, 4, 6, ..., max-2 after this
+    for (long i = 0; i < max; i += 2) {
+      dataJanitorState.savePruneUpperBoundForRegion(Bytes.toBytes(i), i);
+    }
+
+    Assert.assertEquals(10L, dataJanitorState.getPruneUpperBoundForRegion(Bytes.toBytes(10L)));
+
+    // Verify all the saved values
+    for (long i = 0; i < max; ++i) {
+      long expected = i % 2 == 0 ? i : -1;
+      Assert.assertEquals(expected, dataJanitorState.getPruneUpperBoundForRegion(Bytes.toBytes(i)));
+    }
+    // Regions not present should give -1
+    Assert.assertEquals(-1, dataJanitorState.getPruneUpperBoundForRegion(Bytes.toBytes(max + 50L)));
+    Assert.assertEquals(-1, dataJanitorState.getPruneUpperBoundForRegion(Bytes.toBytes((max + 10L) * -1)));
+    Assert.assertEquals(-1, dataJanitorState.getPruneUpperBoundForRegion(Bytes.toBytes(3L)));
+
+    SortedSet<byte[]> allRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    Map<byte[], Long> expectedMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    for (long i = 0; i < max; ++i) {
+      allRegions.add(Bytes.toBytes(i));
+      if (i % 2 == 0) {
+        expectedMap.put(Bytes.toBytes(i), i);
+      }
+    }
+    Assert.assertEquals(max / 2, expectedMap.size());
+    Assert.assertEquals(expectedMap, dataJanitorState.getPruneUpperBoundForRegions(allRegions));
+
+    SortedSet<byte[]> regions = ImmutableSortedSet.orderedBy(Bytes.BYTES_COMPARATOR)
+      .add(Bytes.toBytes((max + 20L) * -1))
+      .add(Bytes.toBytes(6L))
+      .add(Bytes.toBytes(15L))
+      .add(Bytes.toBytes(18L))
+      .add(Bytes.toBytes(max + 33L))
+      .build();
+    expectedMap = ImmutableSortedMap.<byte[], Long>orderedBy(Bytes.BYTES_COMPARATOR)
+      .put(Bytes.toBytes(6L), 6L)
+      .put(Bytes.toBytes(18L), 18L)
+      .build();
+    Assert.assertEquals(expectedMap, dataJanitorState.getPruneUpperBoundForRegions(regions));
+
+    // Delete regions that have prune upper bound before 15 and not in set (4, 8)
+    ImmutableSortedSet<byte[]> excludeRegions =
+      ImmutableSortedSet.orderedBy(Bytes.BYTES_COMPARATOR).add(Bytes.toBytes(4L)).add(Bytes.toBytes(8L)).build();
+    dataJanitorState.deletePruneUpperBounds(15, excludeRegions);
+    // Regions 0, 2, 6 and 10 should have been deleted now
+    expectedMap = ImmutableSortedMap.<byte[], Long>orderedBy(Bytes.BYTES_COMPARATOR)
+      .put(Bytes.toBytes(4L), 4L)
+      .put(Bytes.toBytes(8L), 8L)
+      .put(Bytes.toBytes(16L), 16L)
+      .put(Bytes.toBytes(18L), 18L)
+      .build();
+    Assert.assertEquals(expectedMap, dataJanitorState.getPruneUpperBoundForRegions(allRegions));
+  }
+
+  @Test(timeout = 30000L) // The timeout is used to verify the fix for TEPHRA-230, the test will timeout without the fix
+  public void testSaveRegionTime() throws Exception {
+    int maxTime = 100;
+
+    // Nothing should be present in the beginning
+    Assert.assertNull(dataJanitorState.getRegionsOnOrBeforeTime(maxTime));
+
+    // Save regions for time
+    Map<Long, SortedSet<byte[]>> regionsTime = new TreeMap<>();
+    for (long time = 0; time < maxTime; time += 10) {
+      SortedSet<byte[]> regions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+      for (long region = 0; region < 10; region += 2) {
+        regions.add(Bytes.toBytes((time * 10) + region));
+      }
+      regionsTime.put(time, regions);
+      dataJanitorState.saveRegionsForTime(time, regions);
+    }
+
+    // Verify saved regions
+    Assert.assertEquals(new TimeRegions(0, regionsTime.get(0L)), dataJanitorState.getRegionsOnOrBeforeTime(0));
+    Assert.assertEquals(new TimeRegions(30, regionsTime.get(30L)), dataJanitorState.getRegionsOnOrBeforeTime(30));
+    Assert.assertEquals(new TimeRegions(20, regionsTime.get(20L)), dataJanitorState.getRegionsOnOrBeforeTime(25));
+    Assert.assertEquals(new TimeRegions(30, regionsTime.get(30L)), dataJanitorState.getRegionsOnOrBeforeTime(31));
+    Assert.assertEquals(new TimeRegions(90, regionsTime.get(90L)),
+                        dataJanitorState.getRegionsOnOrBeforeTime(maxTime + 1000));
+    Assert.assertNull(dataJanitorState.getRegionsOnOrBeforeTime(-10));
+
+    // Now change the count stored for regions saved at time 0, 30 and 90
+    try (Table stateTable = testUtil.getConnection().getTable(pruneStateTable)) {
+      dataJanitorState.saveRegionCountForTime(stateTable, Bytes.toBytes(Long.MAX_VALUE), 3);
+      dataJanitorState.saveRegionCountForTime(stateTable, Bytes.toBytes(Long.MAX_VALUE - 30L), 3);
+      dataJanitorState.saveRegionCountForTime(stateTable, Bytes.toBytes(Long.MAX_VALUE - 90L), 0);
+    }
+
+    // Now querying for time 0 should return null, and querying for time 30 should return regions from time 20
+    Assert.assertNull(dataJanitorState.getRegionsOnOrBeforeTime(0));
+    Assert.assertEquals(new TimeRegions(20, regionsTime.get(20L)), dataJanitorState.getRegionsOnOrBeforeTime(30));
+    Assert.assertEquals(new TimeRegions(20, regionsTime.get(20L)), dataJanitorState.getRegionsOnOrBeforeTime(35));
+    Assert.assertEquals(new TimeRegions(20, regionsTime.get(20L)), dataJanitorState.getRegionsOnOrBeforeTime(25));
+    // Querying for anything higher than 90 should give 80 (reproduces TEPHRA-230)
+    Assert.assertEquals(new TimeRegions(80, regionsTime.get(80L)),
+                        dataJanitorState.getRegionsOnOrBeforeTime(Long.MAX_VALUE));
+
+    // Delete regions saved on or before time 30
+    dataJanitorState.deleteAllRegionsOnOrBeforeTime(30);
+    // Values on or before time 30 should be deleted
+    Assert.assertNull(dataJanitorState.getRegionsOnOrBeforeTime(30));
+    Assert.assertNull(dataJanitorState.getRegionsOnOrBeforeTime(25));
+    // Counts should be deleted for time on or before 30
+    try (Table stateTable = testUtil.getConnection().getTable(pruneStateTable)) {
+      Assert.assertEquals(-1, dataJanitorState.getRegionCountForTime(stateTable, 30));
+      Assert.assertEquals(-1, dataJanitorState.getRegionCountForTime(stateTable, 0));
+    }
+    // Values after time 30 should still exist
+    Assert.assertEquals(new TimeRegions(40, regionsTime.get(40L)), dataJanitorState.getRegionsOnOrBeforeTime(40));
+    try (Table stateTable = testUtil.getConnection().getTable(pruneStateTable)) {
+      Assert.assertEquals(5, dataJanitorState.getRegionCountForTime(stateTable, 40));
+    }
+  }
+
+  @Test
+  public void testSaveInactiveTransactionBoundTime() throws Exception {
+    int maxTime = 100;
+
+    // Nothing should be present in the beginning
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime(10));
+
+    // Save inactive transaction bounds for various time values
+    for (long time = 0; time < maxTime; time += 10) {
+      dataJanitorState.saveInactiveTransactionBoundForTime(time, time + 2);
+    }
+
+    // Verify written values
+    Assert.assertEquals(2, dataJanitorState.getInactiveTransactionBoundForTime(0));
+    Assert.assertEquals(12, dataJanitorState.getInactiveTransactionBoundForTime(10));
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime(15));
+    Assert.assertEquals(92, dataJanitorState.getInactiveTransactionBoundForTime(90));
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime(maxTime + 100));
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime((maxTime + 55) * -1L));
+
+    // Delete values saved on or before time 20
+    dataJanitorState.deleteInactiveTransactionBoundsOnOrBeforeTime(20);
+    // Values on or before time 20 should be deleted
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime(0));
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime(10));
+    Assert.assertEquals(-1, dataJanitorState.getInactiveTransactionBoundForTime(20));
+    // Values after time 20 should still exist
+    Assert.assertEquals(32, dataJanitorState.getInactiveTransactionBoundForTime(30));
+    Assert.assertEquals(92, dataJanitorState.getInactiveTransactionBoundForTime(90));
+  }
+
+  @Test
+  public void testSaveEmptyRegions() throws Exception {
+    // Nothing should be present in the beginning
+    Assert.assertEquals(ImmutableSortedSet.<byte[]>of(), dataJanitorState.getEmptyRegionsAfterTime(-1, null));
+
+    byte[] region1 = Bytes.toBytes("region1");
+    byte[] region2 = Bytes.toBytes("region2");
+    byte[] region3 = Bytes.toBytes("region3");
+    byte[] region4 = Bytes.toBytes("region4");
+    SortedSet<byte[]> allRegions = toISet(region1, region2, region3, region4);
+
+    // Now record some empty regions
+    dataJanitorState.saveEmptyRegionForTime(100, region1);
+    dataJanitorState.saveEmptyRegionForTime(110, region1);
+    dataJanitorState.saveEmptyRegionForTime(102, region2);
+    dataJanitorState.saveEmptyRegionForTime(112, region3);
+
+    Assert.assertEquals(toISet(region1, region2, region3),
+                        dataJanitorState.getEmptyRegionsAfterTime(-1, null));
+
+    Assert.assertEquals(toISet(region1, region2, region3),
+                        dataJanitorState.getEmptyRegionsAfterTime(100, allRegions));
+
+    Assert.assertEquals(toISet(region2, region3),
+                        dataJanitorState.getEmptyRegionsAfterTime(100, toISet(region2, region3)));
+
+    Assert.assertEquals(toISet(),
+                        dataJanitorState.getEmptyRegionsAfterTime(100, ImmutableSortedSet.<byte[]>of()));
+
+    Assert.assertEquals(toISet(region3),
+                        dataJanitorState.getEmptyRegionsAfterTime(110, allRegions));
+
+    Assert.assertEquals(toISet(),
+                        dataJanitorState.getEmptyRegionsAfterTime(112, allRegions));
+
+    // Delete empty regions on or before time 110
+    dataJanitorState.deleteEmptyRegionsOnOrBeforeTime(110);
+    // Now only region3 should remain
+    Assert.assertEquals(toISet(region3), dataJanitorState.getEmptyRegionsAfterTime(-1, null));
+    Assert.assertEquals(toISet(region3), dataJanitorState.getEmptyRegionsAfterTime(100, allRegions));
+
+    // Delete empty regions on or before time 150
+    dataJanitorState.deleteEmptyRegionsOnOrBeforeTime(150);
+    // Now nothing should remain
+    Assert.assertEquals(toISet(), dataJanitorState.getEmptyRegionsAfterTime(-1, null));
+  }
+
+  private ImmutableSortedSet<byte[]> toISet(byte[]... args) {
+    ImmutableSortedSet.Builder<byte[]> builder = ImmutableSortedSet.orderedBy(Bytes.BYTES_COMPARATOR);
+    for (byte[] arg : args) {
+      builder.add(arg);
+    }
+    return builder.build();
+  }
+}


[5/5] incubator-tephra git commit: Support for HBase 1.3.x

Posted by an...@apache.org.
Support for HBase 1.3.x

This closes #50 from GitHub.

Signed-off-by: anew <an...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/incubator-tephra/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tephra/commit/8f958edb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tephra/tree/8f958edb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tephra/diff/8f958edb

Branch: refs/heads/master
Commit: 8f958edb6cbd6078ab0d13ccba14467285800a77
Parents: fd6ef73
Author: anew <an...@apache.org>
Authored: Tue Sep 5 13:44:08 2017 -0700
Committer: anew <an...@apache.org>
Committed: Wed Sep 6 00:56:37 2017 -0700

----------------------------------------------------------------------
 pom.xml                                         |    2 +
 .../tephra/util/ConfigurationFactory.java       |    5 +
 .../org/apache/tephra/util/HBaseVersion.java    |    4 +
 .../util/HBaseVersionSpecificFactory.java       |    7 +-
 .../apache/tephra/util/HBaseVersionTest.java    |    3 +
 tephra-examples/hbase-1.3/pom.xml               |  122 ++
 tephra-examples/pom.xml                         |    1 +
 tephra-hbase-compat-1.3/pom.xml                 |  146 ++
 .../hbase/HBase13ConfigurationProvider.java     |   38 +
 .../tephra/hbase/SecondaryIndexTable.java       |  178 ++
 .../tephra/hbase/TransactionAwareHTable.java    |  699 +++++++
 .../hbase/coprocessor/CellSkipFilter.java       |  138 ++
 .../hbase/coprocessor/TransactionFilters.java   |   62 +
 .../hbase/coprocessor/TransactionProcessor.java |  529 ++++++
 .../TransactionVisibilityFilter.java            |  313 ++++
 .../tephra/hbase/txprune/CompactionState.java   |  112 ++
 .../tephra/hbase/txprune/DataJanitorState.java  |  536 ++++++
 .../txprune/HBaseTransactionPruningPlugin.java  |  373 ++++
 .../hbase/txprune/InvalidListPruningDebug.java  |  294 +++
 .../hbase/txprune/PruneUpperBoundWriter.java    |  164 ++
 .../txprune/PruneUpperBoundWriterSupplier.java  |   55 +
 .../tephra/hbase/txprune/TimeRegions.java       |   85 +
 .../tephra/hbase/AbstractHBaseTableTest.java    |  106 ++
 .../hbase/HBase13ConfigurationProviderTest.java |   35 +
 .../hbase/TransactionAwareHTableTest.java       | 1726 ++++++++++++++++++
 .../hbase/coprocessor/CellSkipFilterTest.java   |  123 ++
 .../coprocessor/TransactionProcessorTest.java   |  624 +++++++
 .../TransactionVisibilityFilterTest.java        |  374 ++++
 .../hbase/txprune/DataJanitorStateTest.java     |  285 +++
 .../hbase/txprune/InvalidListPruneTest.java     |  459 +++++
 .../PruneUpperBoundWriterSupplierTest.java      |  122 ++
 31 files changed, 7719 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 79f7a33..71b54ad 100644
--- a/pom.xml
+++ b/pom.xml
@@ -157,6 +157,7 @@
     <module>tephra-hbase-compat-1.0</module>
     <module>tephra-hbase-compat-1.0-cdh</module>
     <module>tephra-hbase-compat-1.1-base</module>
+    <module>tephra-hbase-compat-1.3</module>
     <module>tephra-examples</module>
     <module>tephra-distribution</module>
   </modules>
@@ -212,6 +213,7 @@
     <hbase11.version>1.1.1</hbase11.version>
     <hbase12cdh.version>1.2.0-cdh5.7.0</hbase12cdh.version>
     <hbase12.version>1.2.0</hbase12.version>
+    <hbase13.version>1.3.1</hbase13.version>
     <junit.version>4.11</junit.version>
     <slf4j.version>1.7.5</slf4j.version>
     <thrift.version>0.9.0</thrift.version>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java b/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
index 2703e60..a2eee8a 100644
--- a/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
+++ b/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
@@ -48,6 +48,11 @@ public class ConfigurationFactory implements Provider<Configuration> {
     }
 
     @Override
+    protected String getHBase13Classname() {
+      return "org.apache.tephra.hbase.HBase13ConfigurationProvider";
+    }
+
+    @Override
     protected String getHBase10CDHClassname() {
       return "org.apache.tephra.hbase.HBase10ConfigurationProvider";
     }

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
index e868c6b..814c6f1 100644
--- a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
+++ b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
@@ -37,6 +37,7 @@ public class HBaseVersion {
   private static final String HBASE_10_VERSION = "1.0";
   private static final String HBASE_11_VERSION = "1.1";
   private static final String HBASE_12_VERSION = "1.2";
+  private static final String HBASE_13_VERSION = "1.3";
   private static final String CDH_CLASSIFIER = "cdh";
 
   private static final Logger LOG = LoggerFactory.getLogger(HBaseVersion.class);
@@ -52,6 +53,7 @@ public class HBaseVersion {
     HBASE_10_CDH("1.0-cdh"),
     HBASE_11("1.1"),
     HBASE_12("1.2"),
+    HBASE_13("1.3"),
     UNKNOWN("unknown");
 
     final String majorVersion;
@@ -89,6 +91,8 @@ public class HBaseVersion {
         currentVersion = Version.HBASE_11;
       } else if (versionString.startsWith(HBASE_12_VERSION)) {
         currentVersion = Version.HBASE_12;
+      } else if (versionString.startsWith(HBASE_13_VERSION)) {
+        currentVersion = Version.HBASE_13;
       } else {
         currentVersion = Version.UNKNOWN;
       }

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
index 9153296..ca8db5a 100644
--- a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
+++ b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
@@ -31,7 +31,7 @@ import org.apache.twill.internal.utils.Instances;
 public abstract class HBaseVersionSpecificFactory<T> implements Provider<T> {
   @Override
   public T get() {
-    T instance = null;
+    T instance;
     try {
       switch (HBaseVersion.get()) {
         case HBASE_94:
@@ -52,7 +52,11 @@ public abstract class HBaseVersionSpecificFactory<T> implements Provider<T> {
         case HBASE_12:
           instance = createInstance(getHBase11Classname());
           break;
+        case HBASE_13:
+          instance = createInstance(getHBase13Classname());
+          break;
         case UNKNOWN:
+        default:
           throw new ProvisionException("Unknown HBase version: " + HBaseVersion.getVersionString());
       }
     } catch (ClassNotFoundException cnfe) {
@@ -71,4 +75,5 @@ public abstract class HBaseVersionSpecificFactory<T> implements Provider<T> {
   protected abstract String getHBase10Classname();
   protected abstract String getHBase10CDHClassname();
   protected abstract String getHBase11Classname();
+  protected abstract String getHBase13Classname();
 }

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-core/src/test/java/org/apache/tephra/util/HBaseVersionTest.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/test/java/org/apache/tephra/util/HBaseVersionTest.java b/tephra-core/src/test/java/org/apache/tephra/util/HBaseVersionTest.java
index 27f4032..e114afc 100644
--- a/tephra-core/src/test/java/org/apache/tephra/util/HBaseVersionTest.java
+++ b/tephra-core/src/test/java/org/apache/tephra/util/HBaseVersionTest.java
@@ -84,6 +84,9 @@ public class HBaseVersionTest {
 
     ver = HBaseVersion.VersionNumber.create("1.2.1");
     assertVersionNumber(ver, 1, 2, 1, null, false);
+
+    ver = HBaseVersion.VersionNumber.create("1.3.1");
+    assertVersionNumber(ver, 1, 3, 1, null, false);
   }
 
   private void assertVersionNumber(HBaseVersion.VersionNumber version, Integer expectedMajor, Integer expectedMinor,

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-examples/hbase-1.3/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-1.3/pom.xml b/tephra-examples/hbase-1.3/pom.xml
new file mode 100644
index 0000000..aa9d31b
--- /dev/null
+++ b/tephra-examples/hbase-1.3/pom.xml
@@ -0,0 +1,122 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~      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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <groupId>org.apache.tephra</groupId>
+    <artifactId>tephra-examples</artifactId>
+    <version>0.13.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-1.3</artifactId>
+  <name>Apache Tephra Examples For HBase 1.3</name>
+
+  <properties>
+    <hadoop.version>2.5.1</hadoop.version>
+    <hbase13.version>1.3.1</hbase13.version>
+  </properties>
+
+  <build>
+    <sourceDirectory>../src/main/java</sourceDirectory>
+    <testSourceDirectory>../src/test/java</testSourceDirectory>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tephra</groupId>
+      <artifactId>tephra-hbase-compat-1.3</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase13.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>asm</groupId>
+          <artifactId>asm</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-examples/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/pom.xml b/tephra-examples/pom.xml
index b34a6b6..03c99c0 100644
--- a/tephra-examples/pom.xml
+++ b/tephra-examples/pom.xml
@@ -36,6 +36,7 @@
     <module>hbase-1.0-cdh</module>
     <module>hbase-1.1</module>
     <module>hbase-1.2</module>
+    <module>hbase-1.3</module>
     <module>cdh-5.7</module>
     <module>cdh-5.8</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/pom.xml b/tephra-hbase-compat-1.3/pom.xml
new file mode 100644
index 0000000..ef651a7
--- /dev/null
+++ b/tephra-hbase-compat-1.3/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~      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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <groupId>org.apache.tephra</groupId>
+    <artifactId>tephra</artifactId>
+    <version>0.13.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-hbase-compat-1.3</artifactId>
+  <name>Apache Tephra HBase 1.3 Compatibility</name>
+
+  <properties>
+    <hadoop.version>2.5.1</hadoop.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tephra</groupId>
+      <artifactId>tephra-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tephra</groupId>
+      <artifactId>tephra-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>org.apache.tephra</groupId>
+      <artifactId>tephra-core</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>log4j-over-slf4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>jcl-over-slf4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase13.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <version>${hbase13.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>asm</groupId>
+          <artifactId>asm</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/HBase13ConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/HBase13ConfigurationProvider.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/HBase13ConfigurationProvider.java
new file mode 100644
index 0000000..72efff5
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/HBase13ConfigurationProvider.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.tephra.util.ConfigurationProvider;
+
+/**
+ * HBase 1.3 version of {@link ConfigurationProvider}.
+ */
+public class HBase13ConfigurationProvider extends ConfigurationProvider {
+  @Override
+  public Configuration get() {
+    return HBaseConfiguration.create();
+  }
+
+  @Override
+  public Configuration get(Configuration baseConf) {
+    return HBaseConfiguration.create(baseConf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
new file mode 100644
index 0000000..8bf8768
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase;
+
+import com.google.common.base.Throwables;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+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;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.TransactionContext;
+import org.apache.tephra.TransactionFailureException;
+import org.apache.tephra.distributed.TransactionServiceClient;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A Transactional SecondaryIndexTable.
+ */
+public class SecondaryIndexTable implements Closeable {
+  private byte[] secondaryIndex;
+  private TransactionAwareHTable transactionAwareHTable;
+  private TransactionAwareHTable secondaryIndexTable;
+  private TransactionContext transactionContext;
+  private final TableName secondaryIndexTableName;
+  private static final byte[] secondaryIndexFamily = Bytes.toBytes("secondaryIndexFamily");
+  private static final byte[] secondaryIndexQualifier = Bytes.toBytes('r');
+  private static final byte[] DELIMITER  = new byte[] {0};
+
+  public SecondaryIndexTable(TransactionServiceClient transactionServiceClient, HTableInterface hTable,
+                             byte[] secondaryIndex) {
+    secondaryIndexTableName = TableName.valueOf(hTable.getName().getNameAsString() + ".idx");
+    HTable secondaryIndexHTable = null;
+    try (HBaseAdmin hBaseAdmin = new HBaseAdmin(hTable.getConfiguration())) {
+      if (!hBaseAdmin.tableExists(secondaryIndexTableName)) {
+        hBaseAdmin.createTable(new HTableDescriptor(secondaryIndexTableName));
+      }
+      secondaryIndexHTable = new HTable(hTable.getConfiguration(), secondaryIndexTableName);
+    } catch (Exception e) {
+      Throwables.propagate(e);
+    }
+
+    this.secondaryIndex = secondaryIndex;
+    this.transactionAwareHTable = new TransactionAwareHTable(hTable);
+    this.secondaryIndexTable = new TransactionAwareHTable(secondaryIndexHTable);
+    this.transactionContext = new TransactionContext(transactionServiceClient, transactionAwareHTable,
+                                                     secondaryIndexTable);
+  }
+
+  public Result get(Get get) throws IOException {
+    return get(Collections.singletonList(get))[0];
+  }
+
+  public Result[] get(List<Get> gets) throws IOException {
+    try {
+      transactionContext.start();
+      Result[] result = transactionAwareHTable.get(gets);
+      transactionContext.finish();
+      return result;
+    } catch (Exception e) {
+      try {
+        transactionContext.abort();
+      } catch (TransactionFailureException e1) {
+        throw new IOException("Could not rollback transaction", e1);
+      }
+    }
+    return null;
+  }
+
+  public Result[] getByIndex(byte[] value) throws IOException {
+    try {
+      transactionContext.start();
+      Scan scan = new Scan(value, Bytes.add(value, new byte[0]));
+      scan.addColumn(secondaryIndexFamily, secondaryIndexQualifier);
+      ResultScanner indexScanner = secondaryIndexTable.getScanner(scan);
+
+      ArrayList<Get> gets = new ArrayList<>();
+      for (Result result : indexScanner) {
+        for (Cell cell : result.listCells()) {
+          gets.add(new Get(cell.getValue()));
+        }
+      }
+      Result[] results = transactionAwareHTable.get(gets);
+      transactionContext.finish();
+      return results;
+    } catch (Exception e) {
+      try {
+        transactionContext.abort();
+      } catch (TransactionFailureException e1) {
+        throw new IOException("Could not rollback transaction", e1);
+      }
+    }
+    return null;
+  }
+
+  public void put(Put put) throws IOException {
+    put(Collections.singletonList(put));
+  }
+
+
+  public void put(List<Put> puts) throws IOException {
+    try {
+      transactionContext.start();
+      ArrayList<Put> secondaryIndexPuts = new ArrayList<>();
+      for (Put put : puts) {
+        List<Put> indexPuts = new ArrayList<>();
+        Set<Map.Entry<byte[], List<KeyValue>>> familyMap = put.getFamilyMap().entrySet();
+        for (Map.Entry<byte [], List<KeyValue>> family : familyMap) {
+          for (KeyValue value : family.getValue()) {
+            if (Bytes.equals(value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(),
+                             secondaryIndex, 0, secondaryIndex.length)) {
+              byte[] secondaryRow = Bytes.add(value.getQualifier(), DELIMITER,
+                                                    Bytes.add(value.getValue(), DELIMITER,
+                                                              value.getRow()));
+              Put indexPut = new Put(secondaryRow);
+              indexPut.add(secondaryIndexFamily, secondaryIndexQualifier, put.getRow());
+              indexPuts.add(indexPut);
+            }
+          }
+        }
+        secondaryIndexPuts.addAll(indexPuts);
+      }
+      transactionAwareHTable.put(puts);
+      secondaryIndexTable.put(secondaryIndexPuts);
+      transactionContext.finish();
+    } catch (Exception e) {
+      try {
+        transactionContext.abort();
+      } catch (TransactionFailureException e1) {
+        throw new IOException("Could not rollback transaction", e1);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      transactionAwareHTable.close();
+    } catch (IOException e) {
+      try {
+        secondaryIndexTable.close();
+      } catch (IOException ex) {
+        e.addSuppressed(e);
+      }
+      throw e;
+    }
+    secondaryIndexTable.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
new file mode 100644
index 0000000..ed1dbb1
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
@@ -0,0 +1,699 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.OperationWithAttributes;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.tephra.AbstractTransactionAwareTable;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.TransactionAware;
+import org.apache.tephra.TxConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+
+/**
+ * A Transaction Aware HTable implementation for HBase 1.3. Operations are committed as usual,
+ * but upon a failed or aborted transaction, they are rolled back to the state before the transaction
+ * was started.
+ */
+public class TransactionAwareHTable extends AbstractTransactionAwareTable
+    implements HTableInterface, TransactionAware {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionAwareHTable.class);
+  private final HTableInterface hTable;
+
+  /**
+   * Create a transactional aware instance of the passed HTable
+   *
+   * @param hTable underlying HBase table to use
+   */
+  public TransactionAwareHTable(HTableInterface hTable) {
+    this(hTable, false);
+  }
+
+  /**
+   * Create a transactional aware instance of the passed HTable
+   *
+   * @param hTable underlying HBase table to use
+   * @param conflictLevel level of conflict detection to perform (defaults to {@code COLUMN})
+   */
+  public TransactionAwareHTable(HTableInterface hTable, TxConstants.ConflictDetection conflictLevel) {
+    this(hTable, conflictLevel, false);
+  }
+
+  /**
+   * Create a transactional aware instance of the passed HTable, with the option
+   * of allowing non-transactional operations.
+   * @param hTable underlying HBase table to use
+   * @param allowNonTransactional if true, additional operations (checkAndPut, increment, checkAndDelete)
+   *                              will be available, though non-transactional
+   */
+  public TransactionAwareHTable(HTableInterface hTable, boolean allowNonTransactional) {
+    this(hTable, TxConstants.ConflictDetection.COLUMN, allowNonTransactional);
+  }
+
+  /**
+   * Create a transactional aware instance of the passed HTable, with the option
+   * of allowing non-transactional operations.
+   * @param hTable underlying HBase table to use
+   * @param conflictLevel level of conflict detection to perform (defaults to {@code COLUMN})
+   * @param allowNonTransactional if true, additional operations (checkAndPut, increment, checkAndDelete)
+   *                              will be available, though non-transactional
+   */
+  public TransactionAwareHTable(HTableInterface hTable, TxConstants.ConflictDetection conflictLevel,
+                                boolean allowNonTransactional) {
+    super(conflictLevel, allowNonTransactional);
+    this.hTable = hTable;
+  }
+
+  /* AbstractTransactionAwareTable implementation */
+
+  @Override
+  protected byte[] getTableKey() {
+    return getTableName();
+  }
+
+  @Override
+  protected boolean doCommit() throws IOException {
+    hTable.flushCommits();
+    return true;
+  }
+
+  @Override
+  protected boolean doRollback() throws Exception {
+    try {
+      // pre-size arraylist of deletes
+      int size = 0;
+      for (Set<ActionChange> cs : changeSets.values()) {
+        size += cs.size();
+      }
+      List<Delete> rollbackDeletes = new ArrayList<>(size);
+      for (Map.Entry<Long, Set<ActionChange>> 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);
+          makeRollbackOperation(rollbackDelete);
+          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);
+        }
+      }
+      hTable.delete(rollbackDeletes);
+      return true;
+    } finally {
+      try {
+        hTable.flushCommits();
+      } catch (Exception e) {
+        LOG.error("Could not flush HTable commits", e);
+      }
+      tx = null;
+      changeSets.clear();
+    }
+  }
+
+  /* HTableInterface implementation */
+
+  @Override
+  public byte[] getTableName() {
+    return hTable.getTableName();
+  }
+
+  @Override
+  public TableName getName() {
+    return hTable.getName();
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return hTable.getConfiguration();
+  }
+
+  @Override
+  public HTableDescriptor getTableDescriptor() throws IOException {
+    return hTable.getTableDescriptor();
+  }
+
+  @Override
+  public boolean exists(Get get) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    return hTable.exists(transactionalizeAction(get));
+  }
+
+  @Override
+  public Boolean[] exists(List<Get> gets) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    List<Get> transactionalizedGets = new ArrayList<>(gets.size());
+    for (Get get : gets) {
+      transactionalizedGets.add(transactionalizeAction(get));
+    }
+    return hTable.exists(transactionalizedGets);
+  }
+
+  @Override
+  public void batch(List<? extends Row> actions, Object[] results) throws IOException, InterruptedException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    hTable.batch(transactionalizeActions(actions), results);
+  }
+
+  @Override
+  public Object[] batch(List<? extends Row> actions) throws IOException, InterruptedException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    return hTable.batch(transactionalizeActions(actions));
+  }
+
+  @Override
+  public <R> void batchCallback(List<? extends Row> actions, Object[] results, Batch.Callback<R> callback) throws
+    IOException, InterruptedException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    hTable.batchCallback(transactionalizeActions(actions), results, callback);
+  }
+
+  @Override
+  public <R> Object[] batchCallback(List<? extends Row> actions, Batch.Callback<R> callback) throws IOException,
+    InterruptedException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    return hTable.batchCallback(transactionalizeActions(actions), callback);
+  }
+
+  @Override
+  public Result get(Get get) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    return hTable.get(transactionalizeAction(get));
+  }
+
+  @Override
+  public Result[] get(List<Get> gets) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    ArrayList<Get> transactionalizedGets = new ArrayList<>();
+    for (Get get : gets) {
+      transactionalizedGets.add(transactionalizeAction(get));
+    }
+    return hTable.get(transactionalizedGets);
+  }
+
+  @Override
+  public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.getRowOrBefore(row, family);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public ResultScanner getScanner(Scan scan) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    return hTable.getScanner(transactionalizeAction(scan));
+  }
+
+  @Override
+  public ResultScanner getScanner(byte[] family) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    Scan scan = new Scan();
+    scan.addFamily(family);
+    return hTable.getScanner(transactionalizeAction(scan));
+  }
+
+  @Override
+  public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    Scan scan = new Scan();
+    scan.addColumn(family, qualifier);
+    return hTable.getScanner(transactionalizeAction(scan));
+  }
+
+  @Override
+  public void put(Put put) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    Put txPut = transactionalizeAction(put);
+    hTable.put(txPut);
+  }
+
+  @Override
+  public void put(List<Put> puts) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    List<Put> transactionalizedPuts = new ArrayList<>(puts.size());
+    for (Put put : puts) {
+      Put txPut = transactionalizeAction(put);
+      transactionalizedPuts.add(txPut);
+    }
+    hTable.put(transactionalizedPuts);
+  }
+
+  @Override
+  public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndPut(row, family, qualifier, value, put);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public void delete(Delete delete) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    hTable.delete(transactionalizeAction(delete));
+  }
+
+  @Override
+  public void delete(List<Delete> deletes) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    List<Delete> transactionalizedDeletes = new ArrayList<>(deletes.size());
+    for (Delete delete : deletes) {
+      Delete txDelete = transactionalizeAction(delete);
+      transactionalizedDeletes.add(txDelete);
+    }
+    hTable.delete(transactionalizedDeletes);
+  }
+
+  @Override
+  public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value, Delete delete)
+    throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndDelete(row, family, qualifier, value, delete);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp,
+                                byte[] bytes3, Delete delete) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndDelete(bytes, bytes1, bytes2, compareOp, bytes3, delete);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp,
+                             byte[] bytes3, Put put) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndPut(bytes, bytes1, bytes2, compareOp, bytes3, put);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public boolean[] existsAll(List<Get> gets) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    List<Get> transactionalizedGets = new ArrayList<>(gets.size());
+    for (Get get : gets) {
+      transactionalizedGets.add(transactionalizeAction(get));
+    }
+    return hTable.existsAll(transactionalizedGets);
+  }
+
+  @Override
+  public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
+                                CompareFilter.CompareOp compareOp, byte[] value, RowMutations rowMutations)
+      throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndMutate(row, family, qualifier, compareOp, value, rowMutations);
+    }
+
+    throw new UnsupportedOperationException("checkAndMutate operation is not supported transactionally");
+  }
+
+  @Override
+  public void setOperationTimeout(int operationTimeout) {
+    hTable.setOperationTimeout(operationTimeout);
+  }
+
+  @Override
+  public int getOperationTimeout() {
+    return hTable.getOperationTimeout();
+  }
+
+  @Override
+  public void setRpcTimeout(int rpcTimeout) {
+    hTable.setRpcTimeout(rpcTimeout);
+  }
+
+  @Override
+  public int getRpcTimeout() {
+    return hTable.getRpcTimeout();
+  }
+
+  @Override
+  public void mutateRow(RowMutations rm) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    RowMutations transactionalMutations = new RowMutations();
+    for (Mutation mutation : rm.getMutations()) {
+      if (mutation instanceof Put) {
+        transactionalMutations.add(transactionalizeAction((Put) mutation));
+      } else if (mutation instanceof Delete) {
+        transactionalMutations.add(transactionalizeAction((Delete) mutation));
+      }
+    }
+    hTable.mutateRow(transactionalMutations);
+  }
+
+  @Override
+  public Result append(Append append) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.append(append);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public Result increment(Increment increment) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.increment(increment);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.incrementColumnValue(row, family, qualifier, amount);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability)
+    throws IOException {
+    if (allowNonTransactional) {
+      return hTable.incrementColumnValue(row, family, qualifier, amount, durability);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL)
+    throws IOException {
+    if (allowNonTransactional) {
+      return hTable.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public boolean isAutoFlush() {
+    return hTable.isAutoFlush();
+  }
+
+  @Override
+  public void flushCommits() throws IOException {
+    hTable.flushCommits();
+  }
+
+  @Override
+  public void close() throws IOException {
+    hTable.close();
+  }
+
+  @Override
+  public CoprocessorRpcChannel coprocessorService(byte[] row) {
+    return hTable.coprocessorService(row);
+  }
+
+  @Override
+  public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service, byte[] startKey, byte[] endKey,
+                                                                  Batch.Call<T, R> callable)
+    throws ServiceException, Throwable {
+    return hTable.coprocessorService(service, startKey, endKey, callable);
+  }
+
+  @Override
+  public <T extends Service, R> void coprocessorService(Class<T> service, byte[] startKey, byte[] endKey,
+                                                        Batch.Call<T, R> callable, Batch.Callback<R> callback)
+    throws ServiceException, Throwable {
+    hTable.coprocessorService(service, startKey, endKey, callable, callback);
+  }
+
+  @Override
+  public <R extends Message> Map<byte[], R> batchCoprocessorService(
+      MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey,
+      R responsePrototype) throws ServiceException, Throwable {
+    return hTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype);
+  }
+
+  @Override
+  public <R extends Message> void batchCoprocessorService(MethodDescriptor methodDescriptor,
+      Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback<R> callback)
+      throws ServiceException, Throwable {
+    hTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype, callback);
+  }
+
+  @Override
+  public void setAutoFlush(boolean autoFlush) {
+    setAutoFlushTo(autoFlush);
+  }
+
+  @Override
+  public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
+    hTable.setAutoFlush(autoFlush, clearBufferOnFail);
+  }
+
+  @Override
+  public void setAutoFlushTo(boolean autoFlush) {
+    hTable.setAutoFlushTo(autoFlush);
+  }
+
+  @Override
+  public long getWriteBufferSize() {
+    return hTable.getWriteBufferSize();
+  }
+
+  @Override
+  public void setWriteBufferSize(long writeBufferSize) throws IOException {
+    hTable.setWriteBufferSize(writeBufferSize);
+  }
+
+  // Helpers to get copies of objects with the timestamp set to the current transaction timestamp.
+
+  private Get transactionalizeAction(Get get) throws IOException {
+    addToOperation(get, tx);
+    return get;
+  }
+
+  private Scan transactionalizeAction(Scan scan) throws IOException {
+    addToOperation(scan, tx);
+    return scan;
+  }
+
+  private Put transactionalizeAction(Put put) throws IOException {
+    Put txPut = new Put(put.getRow(), tx.getWritePointer());
+    Set<Map.Entry<byte[], List<Cell>>> familyMap = put.getFamilyCellMap().entrySet();
+    if (!familyMap.isEmpty()) {
+      for (Map.Entry<byte[], List<Cell>> family : familyMap) {
+        List<Cell> familyValues = family.getValue();
+        if (!familyValues.isEmpty()) {
+          for (Cell value : familyValues) {
+            txPut.add(value.getFamily(), value.getQualifier(), tx.getWritePointer(), value.getValue());
+            addToChangeSet(txPut.getRow(), value.getFamily(), value.getQualifier());
+          }
+        }
+      }
+    }
+    for (Map.Entry<String, byte[]> entry : put.getAttributesMap().entrySet()) {
+      txPut.setAttribute(entry.getKey(), entry.getValue());
+    }
+    txPut.setDurability(put.getDurability());
+    addToOperation(txPut, tx);
+    return txPut;
+  }
+
+  private Delete transactionalizeAction(Delete delete) throws IOException {
+    long transactionTimestamp = tx.getWritePointer();
+
+    byte[] deleteRow = delete.getRow();
+    Delete txDelete = new Delete(deleteRow, transactionTimestamp);
+
+    Map<byte[], List<Cell>> familyToDelete = delete.getFamilyCellMap();
+    if (familyToDelete.isEmpty()) {
+      // perform a row delete if we are using row-level conflict detection
+      if (conflictLevel == TxConstants.ConflictDetection.ROW ||
+        conflictLevel == TxConstants.ConflictDetection.NONE) {
+        // Row delete leaves delete markers in all column families of the table
+        // Therefore get all the column families of the hTable from the HTableDescriptor and add them to the changeSet
+        for (HColumnDescriptor columnDescriptor : hTable.getTableDescriptor().getColumnFamilies()) {
+          // no need to identify individual columns deleted
+          addToChangeSet(deleteRow, columnDescriptor.getName(), null);
+        }
+      } else {
+        Result result = get(new Get(delete.getRow()));
+        // Delete everything
+        NavigableMap<byte[], NavigableMap<byte[], byte[]>> resultMap = result.getNoVersionMap();
+        for (Map.Entry<byte[], NavigableMap<byte[], byte[]>> familyEntry : resultMap.entrySet()) {
+          NavigableMap<byte[], byte[]> familyColumns = result.getFamilyMap(familyEntry.getKey());
+          for (Map.Entry<byte[], byte[]> column : familyColumns.entrySet()) {
+            txDelete.deleteColumns(familyEntry.getKey(), column.getKey(), transactionTimestamp);
+            addToChangeSet(deleteRow, familyEntry.getKey(), column.getKey());
+          }
+        }
+      }
+    } else {
+      for (Map.Entry<byte [], List<Cell>> familyEntry : familyToDelete.entrySet()) {
+        byte[] family = familyEntry.getKey();
+        List<Cell> entries = familyEntry.getValue();
+        boolean isFamilyDelete = false;
+        if (entries.size() == 1) {
+          Cell cell = entries.get(0);
+          isFamilyDelete = CellUtil.isDeleteFamily(cell);
+        }
+        if (isFamilyDelete) {
+          if (conflictLevel == TxConstants.ConflictDetection.ROW ||
+              conflictLevel == TxConstants.ConflictDetection.NONE) {
+            // no need to identify individual columns deleted
+            txDelete.deleteFamily(family);
+            addToChangeSet(deleteRow, family, null);
+          } else {
+            Result result = get(new Get(delete.getRow()).addFamily(family));
+            // Delete entire family
+            NavigableMap<byte[], byte[]> familyColumns = result.getFamilyMap(family);
+            for (Map.Entry<byte[], byte[]> column : familyColumns.entrySet()) {
+              txDelete.deleteColumns(family, column.getKey(), transactionTimestamp);
+              addToChangeSet(deleteRow, family, column.getKey());
+            }
+          }
+        } else {
+          for (Cell value : entries) {
+            txDelete.deleteColumns(value.getFamily(), value.getQualifier(), transactionTimestamp);
+            addToChangeSet(deleteRow, value.getFamily(), value.getQualifier());
+          }
+        }
+      }
+    }
+    for (Map.Entry<String, byte[]> entry : delete.getAttributesMap().entrySet()) {
+        txDelete.setAttribute(entry.getKey(), entry.getValue());
+    }
+    txDelete.setDurability(delete.getDurability());
+    addToOperation(txDelete, tx);
+    return txDelete;
+  }
+
+  private List<? extends Row> transactionalizeActions(List<? extends Row> actions) throws IOException {
+    List<Row> transactionalizedActions = new ArrayList<>(actions.size());
+    for (Row action : actions) {
+      if (action instanceof Get) {
+        transactionalizedActions.add(transactionalizeAction((Get) action));
+      } else if (action instanceof Put) {
+        transactionalizedActions.add(transactionalizeAction((Put) action));
+      } else if (action instanceof Delete) {
+        transactionalizedActions.add(transactionalizeAction((Delete) action));
+      } else {
+        transactionalizedActions.add(action);
+      }
+    }
+    return transactionalizedActions;
+  }
+
+  public void addToOperation(OperationWithAttributes op, Transaction tx) throws IOException {
+    op.setAttribute(TxConstants.TX_OPERATION_ATTRIBUTE_KEY, txCodec.encode(tx));
+  }
+
+  protected void makeRollbackOperation(Delete delete) {
+    delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
new file mode 100644
index 0000000..d8664f4
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * {@link Filter} that encapsulates another {@link Filter}. It remembers the last {@link KeyValue}
+ * for which the underlying filter returned the {@link ReturnCode#NEXT_COL} or {@link ReturnCode#INCLUDE_AND_NEXT_COL},
+ * so that when {@link #filterKeyValue} is called again for the same {@link KeyValue} with different
+ * version, it returns {@link ReturnCode#NEXT_COL} directly without consulting the underlying {@link Filter}.
+ * Please see TEPHRA-169 for more details.
+ */
+public class CellSkipFilter extends FilterBase {
+  private final Filter filter;
+  // remember the previous keyvalue processed by filter when the return code was NEXT_COL or INCLUDE_AND_NEXT_COL
+  private KeyValue skipColumn = null;
+
+  public CellSkipFilter(Filter filter) {
+    this.filter = filter;
+  }
+
+  /**
+   * Determines whether the current cell should be skipped. The cell will be skipped
+   * if the previous keyvalue had the same key as the current cell. This means filter already responded
+   * for the previous keyvalue with ReturnCode.NEXT_COL or ReturnCode.INCLUDE_AND_NEXT_COL.
+   * @param cell the {@link Cell} to be tested for skipping
+   * @return true is current cell should be skipped, false otherwise
+   */
+  private boolean skipCellVersion(Cell cell) {
+    return skipColumn != null
+      && CellUtil.matchingRow(cell, skipColumn.getRowArray(), skipColumn.getRowOffset(),
+                              skipColumn.getRowLength())
+      && CellUtil.matchingFamily(cell, skipColumn.getFamilyArray(), skipColumn.getFamilyOffset(),
+                                 skipColumn.getFamilyLength())
+      && CellUtil.matchingQualifier(cell, skipColumn.getQualifierArray(), skipColumn.getQualifierOffset(),
+                                    skipColumn.getQualifierLength());
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(Cell cell) throws IOException {
+    if (skipCellVersion(cell)) {
+      return ReturnCode.NEXT_COL;
+    }
+
+    ReturnCode code = filter.filterKeyValue(cell);
+    if (code == ReturnCode.NEXT_COL || code == ReturnCode.INCLUDE_AND_NEXT_COL) {
+      // only store the reference to the keyvalue if we are returning NEXT_COL or INCLUDE_AND_NEXT_COL
+      skipColumn = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+                                                   cell.getFamilyArray(), cell.getFamilyOffset(),
+                                                   cell.getFamilyLength(), cell.getQualifierArray(),
+                                                   cell.getQualifierOffset(), cell.getQualifierLength());
+    } else {
+      skipColumn = null;
+    }
+    return code;
+  }
+
+  @Override
+  public boolean filterRow() throws IOException {
+    return filter.filterRow();
+  }
+
+  @Override
+  public Cell transformCell(Cell cell) throws IOException {
+    return filter.transformCell(cell);
+  }
+
+  @Override
+  public void reset() throws IOException {
+    filter.reset();
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
+    return filter.filterRowKey(buffer, offset, length);
+  }
+
+  @Override
+  public boolean filterAllRemaining() throws IOException {
+    return filter.filterAllRemaining();
+  }
+
+  @Override
+  public void filterRowCells(List<Cell> kvs) throws IOException {
+    filter.filterRowCells(kvs);
+  }
+
+  @Override
+  public boolean hasFilterRow() {
+    return filter.hasFilterRow();
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
+    return filter.getNextKeyHint(currentKV);
+  }
+
+  @Override
+  public Cell getNextCellHint(Cell currentKV) throws IOException {
+    return filter.getNextCellHint(currentKV);
+  }
+
+  @Override
+  public boolean isFamilyEssential(byte[] name) throws IOException {
+    return filter.isFamilyEssential(name);
+  }
+
+  @Override
+  public byte[] toByteArray() throws IOException {
+    return filter.toByteArray();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
new file mode 100644
index 0000000..0ca9f9c
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.tephra.Transaction;
+
+import java.util.Map;
+import javax.annotation.Nullable;
+
+/**
+ * Factory class for providing {@link Filter} instances.
+ */
+public class TransactionFilters {
+  /**
+   * 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
+   */
+  public static Filter getVisibilityFilter(Transaction tx, Map<byte[], Long> ttlByFamily, boolean allowEmptyValues,
+                                           ScanType scanType) {
+    return new CellSkipFilter(new TransactionVisibilityFilter(tx, ttlByFamily, allowEmptyValues, scanType, null));
+  }
+
+  /**
+   * 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.
+   */
+  public static Filter getVisibilityFilter(Transaction tx, Map<byte[], Long> ttlByFamily, boolean allowEmptyValues,
+                                           ScanType scanType, @Nullable Filter cellFilter) {
+    return new CellSkipFilter(new TransactionVisibilityFilter(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
new file mode 100644
index 0000000..553f598
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
@@ -0,0 +1,529 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.OperationWithAttributes;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.TransactionCodec;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.coprocessor.CacheSupplier;
+import org.apache.tephra.coprocessor.TransactionStateCache;
+import org.apache.tephra.coprocessor.TransactionStateCacheSupplier;
+import org.apache.tephra.hbase.txprune.CompactionState;
+import org.apache.tephra.persist.TransactionVisibilityState;
+import org.apache.tephra.util.TxUtils;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+
+/**
+ * {@code org.apache.hadoop.hbase.coprocessor.RegionObserver} coprocessor that handles server-side processing
+ * for transactions:
+ * <ul>
+ *   <li>applies filtering to exclude data from invalid and in-progress transactions</li>
+ *   <li>overrides the scanner returned for flush and compaction to drop data written by invalidated transactions,
+ *   or expired due to TTL.</li>
+ * </ul>
+ *
+ * <p>In order to use this coprocessor for transactions, configure the class on any table involved in transactions,
+ * or on all user tables by adding the following to hbase-site.xml:
+ * {@code
+ * <property>
+ *   <name>hbase.coprocessor.region.classes</name>
+ *   <value>org.apache.tephra.hbase.coprocessor.TransactionProcessor</value>
+ * </property>
+ * }
+ * </p>
+ *
+ * <p>HBase {@code Get} and {@code Scan} operations should have the current transaction serialized on to the operation
+ * as an attribute:
+ * {@code
+ * Transaction t = ...;
+ * Get get = new Get(...);
+ * TransactionCodec codec = new TransactionCodec();
+ * codec.addToOperation(get, t);
+ * }
+ * </p>
+ */
+public class TransactionProcessor extends BaseRegionObserver {
+  private static final Log LOG = LogFactory.getLog(TransactionProcessor.class);
+
+  private final TransactionCodec txCodec;
+  private TransactionStateCache cache;
+  private volatile CompactionState compactionState;
+  private CacheSupplier<TransactionStateCache> cacheSupplier;
+
+  protected volatile Boolean pruneEnable;
+  protected volatile Long txMaxLifetimeMillis;
+  protected Map<byte[], Long> ttlByFamily = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+  protected boolean allowEmptyValues = TxConstants.ALLOW_EMPTY_VALUES_DEFAULT;
+  protected boolean readNonTxnData = TxConstants.DEFAULT_READ_NON_TX_DATA;
+
+  public TransactionProcessor() {
+    this.txCodec = new TransactionCodec();
+  }
+
+  /* RegionObserver implementation */
+
+  @Override
+  public void start(CoprocessorEnvironment e) throws IOException {
+    if (e instanceof RegionCoprocessorEnvironment) {
+      RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
+      this.cacheSupplier = getTransactionStateCacheSupplier(env);
+      this.cache = cacheSupplier.get();
+
+      HTableDescriptor tableDesc = env.getRegion().getTableDesc();
+      for (HColumnDescriptor columnDesc : tableDesc.getFamilies()) {
+        String columnTTL = columnDesc.getValue(TxConstants.PROPERTY_TTL);
+        long ttl = 0;
+        if (columnTTL != null) {
+          try {
+            ttl = Long.parseLong(columnTTL);
+            LOG.info("Family " + columnDesc.getNameAsString() + " has TTL of " + columnTTL);
+          } catch (NumberFormatException nfe) {
+            LOG.warn("Invalid TTL value configured for column family " + columnDesc.getNameAsString() +
+                       ", value = " + columnTTL);
+          }
+        }
+        ttlByFamily.put(columnDesc.getName(), ttl);
+      }
+
+      this.allowEmptyValues = getAllowEmptyValues(env, tableDesc);
+      this.txMaxLifetimeMillis = getTxMaxLifetimeMillis(env);
+      this.readNonTxnData = Boolean.valueOf(tableDesc.getValue(TxConstants.READ_NON_TX_DATA));
+      if (readNonTxnData) {
+        LOG.info("Reading pre-existing data enabled for table " + tableDesc.getNameAsString());
+      }
+      initializePruneState(env);
+    }
+  }
+
+  /**
+   * Fetch the {@link Configuration} that contains the properties required by the coprocessor. By default,
+   * the HBase configuration is returned. This method will never return {@code null} in Tephra but the derived
+   * classes might do so if {@link Configuration} is not available temporarily (for example, if it is being fetched
+   * from a HBase Table.
+   *
+   * @param env {@link RegionCoprocessorEnvironment} of the Region to which the coprocessor is associated
+   * @return {@link Configuration}, can be null if it is not available
+   */
+  @Nullable
+  protected Configuration getConfiguration(CoprocessorEnvironment env) {
+    return env.getConfiguration();
+  }
+
+  protected CacheSupplier<TransactionStateCache> getTransactionStateCacheSupplier(RegionCoprocessorEnvironment env) {
+    return new TransactionStateCacheSupplier(env.getConfiguration());
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    try {
+      resetPruneState();
+    } finally {
+      if (cacheSupplier != null) {
+        cacheSupplier.release();
+      }
+    }
+  }
+
+  @Override
+  public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
+    throws IOException {
+    Transaction tx = getFromOperation(get);
+    if (tx != null) {
+      projectFamilyDeletes(get);
+      get.setMaxVersions();
+      get.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx, readNonTxnData),
+                       TxUtils.getMaxVisibleTimestamp(tx));
+      Filter newFilter = getTransactionFilter(tx, ScanType.USER_SCAN, get.getFilter());
+      get.setFilter(newFilter);
+    }
+  }
+
+  @Override
+  public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit, Durability durability)
+    throws IOException {
+    Transaction tx = getFromOperation(put);
+    ensureValidTxLifetime(e.getEnvironment(), put, tx);
+  }
+
+  @Override
+  public void preDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete, WALEdit edit,
+                        Durability durability) throws IOException {
+    // Translate deletes into our own delete tombstones
+    // Since HBase deletes cannot be undone, we need to translate deletes into special puts, which allows
+    // us to rollback the changes (by a real delete) if the transaction fails
+
+    // Deletes that are part of a transaction rollback do not need special handling.
+    // They will never be rolled back, so are performed as normal HBase deletes.
+    if (isRollbackOperation(delete)) {
+      return;
+    }
+
+    Transaction tx = getFromOperation(delete);
+    ensureValidTxLifetime(e.getEnvironment(), delete, tx);
+
+    // Other deletes are client-initiated and need to be translated into our own tombstones
+    // TODO: this should delegate to the DeleteStrategy implementation.
+    Put deleteMarkers = new Put(delete.getRow(), delete.getTimeStamp());
+    for (byte[] family : delete.getFamilyCellMap().keySet()) {
+      List<Cell> familyCells = delete.getFamilyCellMap().get(family);
+      if (isFamilyDelete(familyCells)) {
+        deleteMarkers.add(family, TxConstants.FAMILY_DELETE_QUALIFIER, familyCells.get(0).getTimestamp(),
+                          HConstants.EMPTY_BYTE_ARRAY);
+      } else {
+        for (Cell cell : familyCells) {
+          deleteMarkers.add(family, CellUtil.cloneQualifier(cell), cell.getTimestamp(),
+                            HConstants.EMPTY_BYTE_ARRAY);
+        }
+      }
+    }
+    for (Map.Entry<String, byte[]> entry : delete.getAttributesMap().entrySet()) {
+        deleteMarkers.setAttribute(entry.getKey(), entry.getValue());
+    }
+    e.getEnvironment().getRegion().put(deleteMarkers);
+    // skip normal delete handling
+    e.bypass();
+  }
+
+  private boolean getAllowEmptyValues(RegionCoprocessorEnvironment env, HTableDescriptor htd) {
+    String allowEmptyValuesFromTableDesc = htd.getValue(TxConstants.ALLOW_EMPTY_VALUES_KEY);
+    Configuration conf = getConfiguration(env);
+    boolean allowEmptyValuesFromConfig = (conf != null) ?
+      conf.getBoolean(TxConstants.ALLOW_EMPTY_VALUES_KEY, TxConstants.ALLOW_EMPTY_VALUES_DEFAULT) :
+      TxConstants.ALLOW_EMPTY_VALUES_DEFAULT;
+
+    // If the property is not present in the tableDescriptor, get it from the Configuration
+    return  (allowEmptyValuesFromTableDesc != null) ?
+      Boolean.valueOf(allowEmptyValuesFromTableDesc) : allowEmptyValuesFromConfig;
+  }
+
+  private long getTxMaxLifetimeMillis(RegionCoprocessorEnvironment env) {
+    Configuration conf = getConfiguration(env);
+    if (conf != null) {
+      return TimeUnit.SECONDS.toMillis(conf.getInt(TxConstants.Manager.CFG_TX_MAX_LIFETIME,
+                                                   TxConstants.Manager.DEFAULT_TX_MAX_LIFETIME));
+    }
+    return TimeUnit.SECONDS.toMillis(TxConstants.Manager.DEFAULT_TX_MAX_LIFETIME);
+  }
+
+  private boolean isFamilyDelete(List<Cell> familyCells) {
+    return familyCells.size() == 1 && CellUtil.isDeleteFamily(familyCells.get(0));
+  }
+
+  @Override
+  public RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan, RegionScanner s)
+    throws IOException {
+    Transaction tx = getFromOperation(scan);
+    if (tx != null) {
+      projectFamilyDeletes(scan);
+      scan.setMaxVersions();
+      scan.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx, readNonTxnData),
+                        TxUtils.getMaxVisibleTimestamp(tx));
+      Filter newFilter = getTransactionFilter(tx, ScanType.USER_SCAN, scan.getFilter());
+      scan.setFilter(newFilter);
+    }
+    return s;
+  }
+
+  /**
+   * Ensures that family delete markers are present in the columns requested for any scan operation.
+   * @param scan The original scan request
+   * @return The modified scan request with the family delete qualifiers represented
+   */
+  private Scan projectFamilyDeletes(Scan scan) {
+    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
+      NavigableSet<byte[]> columns = entry.getValue();
+      // wildcard scans will automatically include the delete marker, so only need to add it when we have
+      // explicit columns listed
+      if (columns != null && !columns.isEmpty()) {
+        scan.addColumn(entry.getKey(), TxConstants.FAMILY_DELETE_QUALIFIER);
+      }
+    }
+    return scan;
+  }
+
+  /**
+   * Ensures that family delete markers are present in the columns requested for any get operation.
+   * @param get The original get request
+   * @return The modified get request with the family delete qualifiers represented
+   */
+  private Get projectFamilyDeletes(Get get) {
+    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : get.getFamilyMap().entrySet()) {
+      NavigableSet<byte[]> columns = entry.getValue();
+      // wildcard scans will automatically include the delete marker, so only need to add it when we have
+      // explicit columns listed
+      if (columns != null && !columns.isEmpty()) {
+        get.addColumn(entry.getKey(), TxConstants.FAMILY_DELETE_QUALIFIER);
+      }
+    }
+    return get;
+  }
+
+  @Override
+  public InternalScanner preFlushScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+                                             KeyValueScanner memstoreScanner, InternalScanner scanner)
+      throws IOException {
+    return createStoreScanner(c.getEnvironment(), "flush", cache.getLatestState(), store,
+                              Collections.singletonList(memstoreScanner), ScanType.COMPACT_RETAIN_DELETES,
+                              HConstants.OLDEST_TIMESTAMP);
+  }
+
+  @Override
+  public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
+    // Record whether the region is empty after a flush
+    Region region = e.getEnvironment().getRegion();
+    // After a flush, if the memstore size is zero and there are no store files for any stores in the region
+    // then the region must be empty
+    long numStoreFiles = numStoreFilesForRegion(e);
+    long memstoreSize = region.getMemstoreSize();
+    LOG.debug(String.format("Region %s: memstore size = %s, num store files = %s",
+                            region.getRegionInfo().getRegionNameAsString(), memstoreSize, numStoreFiles));
+    if (memstoreSize == 0 && numStoreFiles == 0) {
+      if (compactionState != null) {
+        compactionState.persistRegionEmpty(System.currentTimeMillis());
+      }
+    }
+  }
+
+  @Override
+  public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s,
+      CompactionRequest request)
+      throws IOException {
+    // Get the latest tx snapshot state for the compaction
+    TransactionVisibilityState snapshot = cache.getLatestState();
+
+    // Record tx state before the compaction
+    if (compactionState != null) {
+      compactionState.record(request, snapshot);
+    }
+
+    // Also make sure to use the same snapshot for the compaction
+    return createStoreScanner(c.getEnvironment(), "compaction", snapshot, store, scanners, scanType, earliestPutTs);
+  }
+
+  @Override
+  public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store, StoreFile resultFile,
+                          CompactionRequest request) throws IOException {
+    // Persist the compaction state after a successful compaction
+    if (compactionState != null) {
+      compactionState.persist();
+    }
+  }
+
+  protected InternalScanner createStoreScanner(RegionCoprocessorEnvironment env, String action,
+                                               TransactionVisibilityState snapshot, Store store,
+                                               List<? extends KeyValueScanner> scanners, ScanType type,
+                                               long earliestPutTs) throws IOException {
+    if (snapshot == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Region " + env.getRegion().getRegionInfo().getRegionNameAsString() +
+                    ", no current transaction state found, defaulting to normal " + action + " scanner");
+      }
+      return null;
+    }
+
+    // construct a dummy transaction from the latest snapshot
+    Transaction dummyTx = TxUtils.createDummyTransaction(snapshot);
+    Scan scan = new Scan();
+    // need to see all versions, since we filter out excludes and applications may rely on multiple versions
+    scan.setMaxVersions();
+    scan.setFilter(
+        new IncludeInProgressFilter(dummyTx.getVisibilityUpperBound(),
+            snapshot.getInvalid(),
+            getTransactionFilter(dummyTx, type, null)));
+
+    return new StoreScanner(store, store.getScanInfo(), scan, scanners,
+                            type, store.getSmallestReadPoint(), earliestPutTs);
+  }
+
+  private Transaction getFromOperation(OperationWithAttributes op) throws IOException {
+    byte[] encoded = op.getAttribute(TxConstants.TX_OPERATION_ATTRIBUTE_KEY);
+    if (encoded == null) {
+      // to support old clients
+      encoded = op.getAttribute(TxConstants.OLD_TX_OPERATION_ATTRIBUTE_KEY);
+    }
+    if (encoded != null) {
+      return txCodec.decode(encoded);
+    }
+    return null;
+  }
+
+  /**
+   * Make sure that the transaction is within the max valid transaction lifetime.
+   *
+   * @param env {@link RegionCoprocessorEnvironment} of the Region to which the coprocessor is associated
+   * @param op {@link OperationWithAttributes} HBase operation to access its attributes if required
+   * @param tx {@link Transaction} supplied by the
+   * @throws DoNotRetryIOException thrown if the transaction is older than the max lifetime of a transaction
+   *         IOException throw if the value of max lifetime of transaction is unavailable
+   */
+  protected void ensureValidTxLifetime(RegionCoprocessorEnvironment env,
+                                       @SuppressWarnings("unused") OperationWithAttributes op,
+                                       @Nullable Transaction tx) throws IOException {
+    if (tx == null) {
+      return;
+    }
+
+    boolean validLifetime =
+      (TxUtils.getTimestamp(tx.getTransactionId()) + txMaxLifetimeMillis) > System.currentTimeMillis();
+    if (!validLifetime) {
+      throw new DoNotRetryIOException(String.format("Transaction %s has exceeded max lifetime %s ms",
+                                                    tx.getTransactionId(), txMaxLifetimeMillis));
+    }
+  }
+
+  private boolean isRollbackOperation(OperationWithAttributes op) throws IOException {
+    return op.getAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY) != null ||
+      // to support old clients
+      op.getAttribute(TxConstants.OLD_TX_ROLLBACK_ATTRIBUTE_KEY) != null;
+  }
+
+  /**
+   * Derived classes can override this method to customize the filter used to return data visible for the current
+   * transaction.
+   *
+   * @param tx the current transaction to apply
+   * @param type the type of scan being performed
+   */
+  protected Filter getTransactionFilter(Transaction tx, ScanType type, Filter filter) {
+    return TransactionFilters.getVisibilityFilter(tx, ttlByFamily, allowEmptyValues, type, filter);
+  }
+
+  /**
+   * Refresh the properties related to transaction pruning. This method needs to be invoked if there is change in the
+   * prune related properties after clearing the state by calling {@link #resetPruneState}.
+   *
+   * @param env {@link RegionCoprocessorEnvironment} of this region
+   */
+  protected void initializePruneState(RegionCoprocessorEnvironment env) {
+    Configuration conf = getConfiguration(env);
+    if (conf != null) {
+      pruneEnable = conf.getBoolean(TxConstants.TransactionPruning.PRUNE_ENABLE,
+                                    TxConstants.TransactionPruning.DEFAULT_PRUNE_ENABLE);
+
+      if (Boolean.TRUE.equals(pruneEnable)) {
+        TableName pruneTable = TableName.valueOf(conf.get(TxConstants.TransactionPruning.PRUNE_STATE_TABLE,
+                                                          TxConstants.TransactionPruning.DEFAULT_PRUNE_STATE_TABLE));
+        long pruneFlushInterval = TimeUnit.SECONDS.toMillis(conf.getLong(
+          TxConstants.TransactionPruning.PRUNE_FLUSH_INTERVAL,
+          TxConstants.TransactionPruning.DEFAULT_PRUNE_FLUSH_INTERVAL));
+
+        compactionState = new CompactionState(env, pruneTable, pruneFlushInterval);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("Automatic invalid list pruning is enabled for table %s. Compaction state " +
+                                    "will be recorded in table %s",
+                                  env.getRegionInfo().getTable().getNameWithNamespaceInclAsString(),
+                                  pruneTable.getNameWithNamespaceInclAsString()));
+        }
+      }
+    }
+  }
+
+  /**
+   * Stop and clear state related to pruning.
+   */
+  protected void resetPruneState() {
+    pruneEnable = false;
+    if (compactionState != null) {
+      compactionState.stop();
+      compactionState = null;
+    }
+  }
+
+  private long numStoreFilesForRegion(ObserverContext<RegionCoprocessorEnvironment> c) {
+    long numStoreFiles = 0;
+    for (Store store : c.getEnvironment().getRegion().getStores()) {
+      numStoreFiles += store.getStorefiles().size();
+    }
+    return numStoreFiles;
+  }
+
+  /**
+   * Filter used to include cells visible to in-progress transactions on flush and commit.
+   */
+  static class IncludeInProgressFilter extends FilterBase {
+    private final long visibilityUpperBound;
+    private final Set<Long> invalidIds;
+    private final Filter txFilter;
+
+    public IncludeInProgressFilter(long upperBound, Collection<Long> invalids, Filter transactionFilter) {
+      this.visibilityUpperBound = upperBound;
+      this.invalidIds = Sets.newHashSet(invalids);
+      this.txFilter = transactionFilter;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(Cell cell) throws IOException {
+      // include all cells visible to in-progress transactions, except for those already marked as invalid
+      long ts = cell.getTimestamp();
+      if (ts > visibilityUpperBound) {
+        // include everything that could still be in-progress except invalids
+        if (invalidIds.contains(ts)) {
+          return ReturnCode.SKIP;
+        }
+        return ReturnCode.INCLUDE;
+      }
+      return txFilter.filterKeyValue(cell);
+    }
+  }
+}



[4/5] incubator-tephra git commit: Support for HBase 1.3.x

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
new file mode 100644
index 0000000..40e2c37
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.coprocessor;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.util.TxUtils;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+
+/**
+ * Applies filtering of data based on transactional visibility (HBase 1.3 specific version).
+ * Note: this is intended for server-side use only, as additional properties need to be set on
+ * any {@code Scan} or {@code Get} operation performed.
+ */
+public class TransactionVisibilityFilter extends FilterBase {
+  private final Transaction tx;
+  // oldest visible timestamp by column family, used to apply TTL when reading
+  private final Map<ImmutableBytesWritable, Long> oldestTsByFamily;
+  // if false, empty values will be interpreted as deletes
+  private final boolean allowEmptyValues;
+  // whether or not we can remove delete markers
+  // these can only be safely removed when we are traversing all storefiles
+  private final boolean clearDeletes;
+  // optional sub-filter to apply to visible cells
+  private final Filter cellFilter;
+  // since we traverse KVs in order, cache the current oldest TS to avoid map lookups per KV
+  private final ImmutableBytesWritable currentFamily = new ImmutableBytesWritable(HConstants.EMPTY_BYTE_ARRAY);
+  
+  private long currentOldestTs;
+
+  private DeleteTracker deleteTracker = new DeleteTracker();
+
+  /**
+   * 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
+   */
+  public TransactionVisibilityFilter(Transaction tx, Map<byte[], Long> ttlByFamily, boolean allowEmptyValues,
+                              ScanType scanType) {
+    this(tx, ttlByFamily, allowEmptyValues, scanType, null);
+  }
+
+  /**
+   * 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.
+   */
+   public TransactionVisibilityFilter(Transaction tx, Map<byte[], Long> ttlByFamily, boolean allowEmptyValues,
+                               ScanType scanType, @Nullable Filter cellFilter) {
+    this.tx = tx;
+    this.oldestTsByFamily = Maps.newTreeMap();
+    for (Map.Entry<byte[], Long> ttlEntry : ttlByFamily.entrySet()) {
+      long familyTTL = ttlEntry.getValue();
+      oldestTsByFamily.put(new ImmutableBytesWritable(ttlEntry.getKey()),
+                           familyTTL <= 0 ? 0 : tx.getVisibilityUpperBound() - familyTTL * TxConstants.MAX_TX_PER_MS);
+    }
+    this.allowEmptyValues = allowEmptyValues;
+    this.clearDeletes =
+      scanType == ScanType.COMPACT_DROP_DELETES ||
+        (scanType == ScanType.USER_SCAN && tx.getVisibilityLevel() != Transaction.VisibilityLevel.SNAPSHOT_ALL);
+    this.cellFilter = cellFilter;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(Cell cell) throws IOException {
+    if (!CellUtil.matchingFamily(cell, currentFamily.get(), currentFamily.getOffset(), currentFamily.getLength())) {
+      // column family changed
+      currentFamily.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
+      Long familyOldestTs = oldestTsByFamily.get(currentFamily);
+      currentOldestTs = familyOldestTs != null ? familyOldestTs : 0;
+      deleteTracker.reset();
+    }
+    // need to apply TTL for the column family here
+    long kvTimestamp = cell.getTimestamp();
+    if (TxUtils.getTimestampForTTL(kvTimestamp) < currentOldestTs) {
+      // passed TTL for this column, seek to next
+      return ReturnCode.NEXT_COL;
+    } else if (tx.isVisible(kvTimestamp)) {
+      // Return all writes done by current transaction (including deletes) for VisibilityLevel.SNAPSHOT_ALL
+      if (tx.getVisibilityLevel() == Transaction.VisibilityLevel.SNAPSHOT_ALL && tx.isCurrentWrite(kvTimestamp)) {
+        // cell is visible
+        // visibility SNAPSHOT_ALL needs all matches
+        return runSubFilter(ReturnCode.INCLUDE, cell);
+      }
+      if (DeleteTracker.isFamilyDelete(cell)) {
+        deleteTracker.addFamilyDelete(cell);
+        if (clearDeletes) {
+          return ReturnCode.NEXT_COL;
+        } else {
+          // cell is visible
+          // as soon as we find a KV to include we can move to the next column
+          return runSubFilter(ReturnCode.INCLUDE_AND_NEXT_COL, cell);
+        }
+      }
+      // check if masked by family delete
+      if (deleteTracker.isDeleted(cell)) {
+        return ReturnCode.NEXT_COL;
+      }
+      // check for column delete
+      if (isColumnDelete(cell)) {
+        if (clearDeletes) {
+          // skip "deleted" cell
+          return ReturnCode.NEXT_COL;
+        } else {
+          // keep the marker but skip any remaining versions
+          return runSubFilter(ReturnCode.INCLUDE_AND_NEXT_COL, cell);
+        }
+      }
+      // cell is visible
+      // as soon as we find a KV to include we can move to the next column
+      return runSubFilter(ReturnCode.INCLUDE_AND_NEXT_COL, cell);
+    } else {
+      return ReturnCode.SKIP;
+    }
+  }
+
+  private ReturnCode runSubFilter(ReturnCode txFilterCode, Cell cell) throws IOException {
+    if (cellFilter != null) {
+      ReturnCode subFilterCode = cellFilter.filterKeyValue(cell);
+      return determineReturnCode(txFilterCode, subFilterCode);
+    }
+    return txFilterCode;
+  }
+
+  /**
+   * Determines the return code of TransactionVisibilityFilter based on sub-filter's return code.
+   * Sub-filter can only exclude cells included by TransactionVisibilityFilter, i.e., sub-filter's
+   * INCLUDE will be ignored. This behavior makes sure that sub-filter only sees cell versions valid for the
+   * given transaction. If sub-filter needs to see older versions of cell, then this method can be overridden.
+   *
+   * @param txFilterCode return code from TransactionVisibilityFilter
+   * @param subFilterCode return code from sub-filter
+   * @return final return code
+   */
+  protected ReturnCode determineReturnCode(ReturnCode txFilterCode, ReturnCode subFilterCode) {
+    // Return the more restrictive of the two filter responses
+    switch (subFilterCode) {
+      case INCLUDE:
+        return txFilterCode;
+      case INCLUDE_AND_NEXT_COL:
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      case SKIP:
+        return txFilterCode == ReturnCode.INCLUDE ? ReturnCode.SKIP : ReturnCode.NEXT_COL;
+      default:
+        return subFilterCode;
+    }
+  }
+
+  @Override
+  public boolean filterRow() throws IOException {
+    if (cellFilter != null) {
+      return cellFilter.filterRow();
+    }
+    return super.filterRow();
+  }
+  
+  @Override
+  public Cell transformCell(Cell cell) throws IOException {
+    // Convert Tephra deletes back into HBase deletes
+    if (tx.getVisibilityLevel() == Transaction.VisibilityLevel.SNAPSHOT_ALL) {
+      if (DeleteTracker.isFamilyDelete(cell)) {
+        return new KeyValue(CellUtil.cloneRow(cell), CellUtil.cloneFamily(cell), null, cell.getTimestamp(),
+                            KeyValue.Type.DeleteFamily);
+      } else if (isColumnDelete(cell)) {
+        // Note: in some cases KeyValue.Type.Delete is used in Delete object,
+        // and in some other cases KeyValue.Type.DeleteColumn is used.
+        // Since Tephra cannot distinguish between the two, we return KeyValue.Type.DeleteColumn.
+        // KeyValue.Type.DeleteColumn makes both CellUtil.isDelete and CellUtil.isDeleteColumns return true, and will
+        // work in both cases.
+        return new KeyValue(CellUtil.cloneRow(cell), CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell),
+                            cell.getTimestamp(), KeyValue.Type.DeleteColumn);
+      }
+    }
+    return cell;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    deleteTracker.reset();
+    if (cellFilter != null) {
+      cellFilter.reset();
+    }
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
+    if (cellFilter != null) {
+      return cellFilter.filterRowKey(buffer, offset, length);
+    }
+    return super.filterRowKey(buffer, offset, length);
+  }
+
+  @Override
+  public boolean filterAllRemaining() throws IOException {
+    if (cellFilter != null) {
+      return cellFilter.filterAllRemaining();
+    }
+    return super.filterAllRemaining();
+  }
+
+  @Override
+  public void filterRowCells(List<Cell> kvs) throws IOException {
+    if (cellFilter != null) {
+      cellFilter.filterRowCells(kvs);
+    } else {
+      super.filterRowCells(kvs);
+    }
+  }
+
+  @Override
+  public boolean hasFilterRow() {
+    if (cellFilter != null) {
+      return cellFilter.hasFilterRow();
+    }
+    return super.hasFilterRow();
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
+    if (cellFilter != null) {
+      return cellFilter.getNextKeyHint(currentKV);
+    }
+    return super.getNextKeyHint(currentKV);
+  }
+
+  @Override
+  public Cell getNextCellHint(Cell currentKV) throws IOException {
+    if (cellFilter != null) {
+      return cellFilter.getNextCellHint(currentKV);
+    }
+    return super.getNextCellHint(currentKV);
+  }
+
+  @Override
+  public boolean isFamilyEssential(byte[] name) throws IOException {
+    if (cellFilter != null) {
+      return cellFilter.isFamilyEssential(name);
+    }
+    return super.isFamilyEssential(name);
+  }
+
+  private boolean isColumnDelete(Cell cell) {
+    return !TxUtils.isPreExistingVersion(cell.getTimestamp()) && cell.getValueLength() == 0 && !allowEmptyValues;
+  }
+
+  private static final class DeleteTracker {
+    private long familyDeleteTs;
+    private byte[] rowKey;
+
+    public static boolean isFamilyDelete(Cell cell) {
+      return !TxUtils.isPreExistingVersion(cell.getTimestamp()) &&
+        CellUtil.matchingQualifier(cell, TxConstants.FAMILY_DELETE_QUALIFIER) &&
+        CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY);
+    }
+
+    public void addFamilyDelete(Cell delete) {
+      this.familyDeleteTs = delete.getTimestamp();
+      this.rowKey = Bytes.copy(delete.getRowArray(), delete.getRowOffset(), delete.getRowLength());
+    }
+
+    public boolean isDeleted(Cell cell) {
+      return rowKey != null && Bytes.compareTo(cell.getRowArray(), cell.getRowOffset(), 
+        cell.getRowLength(), rowKey, 0, rowKey.length) == 0 && cell.getTimestamp() <= familyDeleteTs;
+    }
+
+    public void reset() {
+      this.familyDeleteTs = 0;
+      this.rowKey = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java
new file mode 100644
index 0000000..9b856d9
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.persist.TransactionVisibilityState;
+import org.apache.tephra.util.TxUtils;
+
+import java.io.IOException;
+import javax.annotation.Nullable;
+
+/**
+ * Record compaction state for invalid list pruning
+ */
+public class CompactionState {
+  private static final Log LOG = LogFactory.getLog(CompactionState.class);
+
+  private final byte[] regionName;
+  private final String regionNameAsString;
+  private final PruneUpperBoundWriterSupplier pruneUpperBoundWriterSupplier;
+  private final PruneUpperBoundWriter pruneUpperBoundWriter;
+
+  private volatile long pruneUpperBound = -1;
+
+  public CompactionState(final RegionCoprocessorEnvironment env, final TableName stateTable, long pruneFlushInterval) {
+    this.regionName = env.getRegionInfo().getRegionName();
+    this.regionNameAsString = env.getRegionInfo().getRegionNameAsString();
+    DataJanitorState dataJanitorState = new DataJanitorState(new DataJanitorState.TableSupplier() {
+      @Override
+      public Table get() throws IOException {
+        return env.getTable(stateTable);
+      }
+    });
+    this.pruneUpperBoundWriterSupplier = new PruneUpperBoundWriterSupplier(stateTable, dataJanitorState,
+                                                                           pruneFlushInterval);
+    this.pruneUpperBoundWriter = pruneUpperBoundWriterSupplier.get();
+  }
+
+  /**
+   * Records the transaction state used for a compaction. This method is called when the compaction starts.
+   *
+   * @param request {@link CompactionRequest} for the compaction
+   * @param snapshot transaction state that will be used for the compaction
+   */
+  public void record(CompactionRequest request, @Nullable TransactionVisibilityState snapshot) {
+    if (request.isMajor() && snapshot != null) {
+      Transaction tx = TxUtils.createDummyTransaction(snapshot);
+      pruneUpperBound = TxUtils.getPruneUpperBound(tx);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+          String.format("Computed prune upper bound %s for compaction request %s using transaction state from time %s",
+                        pruneUpperBound, request, snapshot.getTimestamp()));
+      }
+    } else {
+      pruneUpperBound = -1;
+    }
+  }
+
+  /**
+   * Persists the transaction state recorded by {@link #record(CompactionRequest, TransactionVisibilityState)}.
+   * This method is called after the compaction has successfully completed.
+   */
+  public void persist() {
+    if (pruneUpperBound != -1) {
+      pruneUpperBoundWriter.persistPruneEntry(regionName, pruneUpperBound);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Enqueued prune upper bound %s for region %s", pruneUpperBound, regionNameAsString));
+      }
+    }
+  }
+
+  /**
+   * Persist that the given region is empty at the given time
+   * @param time time in milliseconds
+   */
+  public void persistRegionEmpty(long time) {
+    pruneUpperBoundWriter.persistRegionEmpty(regionName, time);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("Enqueued empty region %s at time %s", regionNameAsString, time));
+    }
+  }
+
+  /**
+   * Releases the usage {@link PruneUpperBoundWriter}.
+   */
+  public void stop() {
+    pruneUpperBoundWriterSupplier.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java
new file mode 100644
index 0000000..db59d7d
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java
@@ -0,0 +1,536 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+import org.apache.tephra.txprune.RegionPruneInfo;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import javax.annotation.Nullable;
+
+/**
+ * Persist data janitor state into an HBase table.
+ * This is used by both {@link TransactionProcessor} and by the {@link HBaseTransactionPruningPlugin}
+ * to persist and read the compaction state.
+ */
+@SuppressWarnings("WeakerAccess")
+public class DataJanitorState {
+  private static final Log LOG = LogFactory.getLog(DataJanitorState.class);
+
+  public static final byte[] FAMILY = {'f'};
+  public static final byte[] PRUNE_UPPER_BOUND_COL = {'p'};
+
+  private static final byte[] REGION_TIME_COL = {'r'};
+  private static final byte[] INACTIVE_TRANSACTION_BOUND_TIME_COL = {'i'};
+  private static final byte[] EMPTY_REGION_TIME_COL = {'e'};
+
+  private static final byte[] REGION_KEY_PREFIX = {0x1};
+  private static final byte[] REGION_KEY_PREFIX_STOP = {0x2};
+
+  private static final byte[] REGION_TIME_KEY_PREFIX = {0x2};
+  private static final byte[] REGION_TIME_KEY_PREFIX_STOP = {0x3};
+
+  private static final byte[] INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX = {0x3};
+  private static final byte[] INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX_STOP = {0x4};
+
+  private static final byte[] EMPTY_REGION_TIME_KEY_PREFIX = {0x4};
+  private static final byte[] EMPTY_REGION_TIME_KEY_PREFIX_STOP = {0x5};
+
+  private static final byte[] REGION_TIME_COUNT_KEY_PREFIX = {0x5};
+  private static final byte[] REGION_TIME_COUNT_KEY_PREFIX_STOP = {0x6};
+
+  private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
+  // This value can be used when we don't care about the value we write in a column
+  private static final byte[] COL_VAL = Bytes.toBytes('1');
+
+  private final TableSupplier stateTableSupplier;
+
+
+  public DataJanitorState(TableSupplier stateTableSupplier) {
+    this.stateTableSupplier = stateTableSupplier;
+  }
+
+  // ----------------------------------------------------------------
+  // ------- Methods for prune upper bound for a given region -------
+  // ----------------------------------------------------------------
+  // The data is stored in the following format -
+  // Key: 0x1<region-id>
+  // Col 'u': <prune upper bound>
+  // ----------------------------------------------------------------
+
+  /**
+   * Persist the latest prune upper bound for a given region. This is called by {@link TransactionProcessor}
+   * after major compaction.
+   *
+   * @param regionId region id
+   * @param pruneUpperBound the latest prune upper bound for the region
+   * @throws IOException when not able to persist the data to HBase
+   */
+  public void savePruneUpperBoundForRegion(byte[] regionId, long pruneUpperBound) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      Put put = new Put(makeRegionKey(regionId));
+      put.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL, Bytes.toBytes(pruneUpperBound));
+      stateTable.put(put);
+    }
+  }
+
+  /**
+   * Get latest prune upper bound for a given region. This indicates the largest invalid transaction that no
+   * longer has writes in this region.
+   *
+   * @param regionId region id
+   * @return latest prune upper bound for the region
+   * @throws IOException when not able to read the data from HBase
+   */
+  public long getPruneUpperBoundForRegion(byte[] regionId) throws IOException {
+    RegionPruneInfo regionPruneInfo = getPruneInfoForRegion(regionId);
+    return (regionPruneInfo == null) ? -1 : regionPruneInfo.getPruneUpperBound();
+  }
+
+  /**
+   * Get the latest {@link RegionPruneInfo} for a given region.
+   *
+   * @param regionId region id
+   * @return {@link RegionPruneInfo} for the region
+   * @throws IOException when not able to read the data from HBase
+   */
+  @Nullable
+  public RegionPruneInfo getPruneInfoForRegion(byte[] regionId) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      Get get = new Get(makeRegionKey(regionId));
+      get.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL);
+      Cell cell = stateTable.get(get).getColumnLatestCell(FAMILY, PRUNE_UPPER_BOUND_COL);
+      if (cell == null) {
+        return null;
+      }
+      byte[] pruneUpperBoundBytes = CellUtil.cloneValue(cell);
+      long timestamp = cell.getTimestamp();
+      return new RegionPruneInfo(regionId, Bytes.toStringBinary(regionId),
+                                 Bytes.toLong(pruneUpperBoundBytes), timestamp);
+    }
+  }
+
+  /**
+   * Get latest prune upper bounds for given regions. This is a batch operation of method
+   * {@link #getPruneUpperBoundForRegion(byte[])}
+   *
+   * @param regions a set of regions
+   * @return a map containing region id and its latest prune upper bound value
+   * @throws IOException when not able to read the data from HBase
+   */
+  public Map<byte[], Long> getPruneUpperBoundForRegions(SortedSet<byte[]> regions) throws IOException {
+    Map<byte[], Long> resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    List<RegionPruneInfo> regionPruneInfos = getPruneInfoForRegions(regions);
+    for (RegionPruneInfo regionPruneInfo : regionPruneInfos) {
+      resultMap.put(regionPruneInfo.getRegionName(), regionPruneInfo.getPruneUpperBound());
+    }
+    return Collections.unmodifiableMap(resultMap);
+  }
+
+  /**
+   * Gets a list of {@link RegionPruneInfo} for given regions. Returns all regions if the given regions set is null.
+   *
+   * @param regions a set of regions
+   * @return list of {@link RegionPruneInfo}s.
+   * @throws IOException when not able to read the data from HBase
+   */
+  public List<RegionPruneInfo> getPruneInfoForRegions(@Nullable SortedSet<byte[]> regions) throws IOException {
+    List<RegionPruneInfo> regionPruneInfos = new ArrayList<>();
+    try (Table stateTable = stateTableSupplier.get()) {
+      byte[] startRow = makeRegionKey(EMPTY_BYTE_ARRAY);
+      Scan scan = new Scan(startRow, REGION_KEY_PREFIX_STOP);
+      scan.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL);
+
+      try (ResultScanner scanner = stateTable.getScanner(scan)) {
+        Result next;
+        while ((next = scanner.next()) != null) {
+          byte[] region = getRegionFromKey(next.getRow());
+          if (regions == null || regions.contains(region)) {
+            Cell cell = next.getColumnLatestCell(FAMILY, PRUNE_UPPER_BOUND_COL);
+            if (cell != null) {
+              byte[] pruneUpperBoundBytes = CellUtil.cloneValue(cell);
+              long timestamp = cell.getTimestamp();
+              regionPruneInfos.add(new RegionPruneInfo(region, Bytes.toStringBinary(region),
+                                                       Bytes.toLong(pruneUpperBoundBytes), timestamp));
+            }
+          }
+        }
+      }
+    }
+    return Collections.unmodifiableList(regionPruneInfos);
+  }
+
+  /**
+   * Delete prune upper bounds for the regions that are not in the given exclude set, and the
+   * prune upper bound is less than the given value.
+   * After the invalid list is pruned up to deletionPruneUpperBound, we do not need entries for regions that have
+   * prune upper bound less than deletionPruneUpperBound. We however limit the deletion to only regions that are
+   * no longer in existence (due to deletion, etc.), to avoid update/delete race conditions.
+   *
+   * @param deletionPruneUpperBound prune upper bound below which regions will be deleted
+   * @param excludeRegions set of regions that should not be deleted
+   * @throws IOException when not able to delete data in HBase
+   */
+  public void deletePruneUpperBounds(long deletionPruneUpperBound, SortedSet<byte[]> excludeRegions)
+    throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      byte[] startRow = makeRegionKey(EMPTY_BYTE_ARRAY);
+      Scan scan = new Scan(startRow, REGION_KEY_PREFIX_STOP);
+      scan.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL);
+
+      try (ResultScanner scanner = stateTable.getScanner(scan)) {
+        Result next;
+        while ((next = scanner.next()) != null) {
+          byte[] region = getRegionFromKey(next.getRow());
+          if (!excludeRegions.contains(region)) {
+            byte[] timeBytes = next.getValue(FAMILY, PRUNE_UPPER_BOUND_COL);
+            if (timeBytes != null) {
+              long pruneUpperBoundRegion = Bytes.toLong(timeBytes);
+              if (pruneUpperBoundRegion < deletionPruneUpperBound) {
+                stateTable.delete(new Delete(next.getRow()));
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  // ---------------------------------------------------
+  // ------- Methods for regions at a given time -------
+  // ---------------------------------------------------
+  // Key: 0x2<inverted time><region-id>
+  // Col 't': <empty byte array>
+  // ---------------------------------------------------
+
+  /**
+   * Persist the regions for the given time. {@link HBaseTransactionPruningPlugin} saves the set of
+   * transactional regions existing in the HBase instance periodically.
+   *
+   * @param time timestamp in milliseconds
+   * @param regions set of regions at the time
+   * @throws IOException when not able to persist the data to HBase
+   */
+  public void saveRegionsForTime(long time, Set<byte[]> regions) throws IOException {
+    byte[] timeBytes = Bytes.toBytes(getInvertedTime(time));
+    try (Table stateTable = stateTableSupplier.get()) {
+      for (byte[] region : regions) {
+        Put put = new Put(makeTimeRegionKey(timeBytes, region));
+        put.addColumn(FAMILY, REGION_TIME_COL, COL_VAL);
+        stateTable.put(put);
+      }
+
+      // Save the count of regions as a checksum
+      saveRegionCountForTime(stateTable, timeBytes, regions.size());
+    }
+  }
+
+  @VisibleForTesting
+  void saveRegionCountForTime(Table stateTable, byte[] timeBytes, int count) throws IOException {
+    Put put = new Put(makeTimeRegionCountKey(timeBytes));
+    put.addColumn(FAMILY, REGION_TIME_COL, Bytes.toBytes(count));
+    stateTable.put(put);
+  }
+
+  /**
+   * Return the set of regions saved for the time at or before the given time. This method finds the greatest time
+   * that is less than or equal to the given time, and then returns all regions with that exact time, but none that are
+   * older than that.
+   *
+   * @param time timestamp in milliseconds
+   * @return set of regions and time at which they were recorded, or null if no regions found
+   * @throws IOException when not able to read the data from HBase
+   */
+  @Nullable
+  public TimeRegions getRegionsOnOrBeforeTime(long time) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      TimeRegions timeRegions;
+      while ((timeRegions = getNextSetOfTimeRegions(stateTable, time)) != null) {
+        int count = getRegionCountForTime(stateTable, timeRegions.getTime());
+        if (count != -1 && count == timeRegions.getRegions().size()) {
+          return timeRegions;
+        } else {
+          LOG.warn(String.format("Got incorrect count for regions saved at time %s, expected = %s but actual = %s",
+                                 timeRegions.getTime(), count, timeRegions.getRegions().size()));
+          time = timeRegions.getTime() - 1;
+        }
+      }
+      return null;
+    }
+  }
+
+  @Nullable
+  private TimeRegions getNextSetOfTimeRegions(Table stateTable, long time) throws IOException {
+    byte[] timeBytes = Bytes.toBytes(getInvertedTime(time));
+    Scan scan = new Scan(makeTimeRegionKey(timeBytes, EMPTY_BYTE_ARRAY), REGION_TIME_KEY_PREFIX_STOP);
+    scan.addColumn(FAMILY, REGION_TIME_COL);
+
+
+    long currentRegionTime = -1;
+    SortedSet<byte[]> regions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    Result next;
+    try (ResultScanner scanner = stateTable.getScanner(scan)) {
+      while ((next = scanner.next()) != null) {
+        Map.Entry<Long, byte[]> timeRegion = getTimeRegion(next.getRow());
+        // Stop if reached next time value
+        if (currentRegionTime == -1) {
+          currentRegionTime = timeRegion.getKey();
+        } else if (timeRegion.getKey() < currentRegionTime) {
+          break;
+        } else if (timeRegion.getKey() > currentRegionTime) {
+          throw new IllegalStateException(
+            String.format("Got out of order time %d when expecting time less than or equal to %d",
+                          timeRegion.getKey(), currentRegionTime));
+        }
+        regions.add(timeRegion.getValue());
+      }
+    }
+    return regions.isEmpty() ? null : new TimeRegions(currentRegionTime, Collections.unmodifiableSortedSet(regions));
+  }
+
+  @VisibleForTesting
+  int getRegionCountForTime(Table stateTable, long time) throws IOException {
+    Get get = new Get(makeTimeRegionCountKey(Bytes.toBytes(getInvertedTime(time))));
+    get.addColumn(FAMILY, REGION_TIME_COL);
+    Result result = stateTable.get(get);
+    byte[] value = result.getValue(FAMILY, REGION_TIME_COL);
+    return value == null ? -1 : Bytes.toInt(value);
+  }
+
+  /**
+   * Delete all the regions that were recorded for all times equal or less than the given time.
+   *
+   * @param time timestamp in milliseconds
+   * @throws IOException when not able to delete data in HBase
+   */
+  public void deleteAllRegionsOnOrBeforeTime(long time) throws IOException {
+    byte[] timeBytes = Bytes.toBytes(getInvertedTime(time));
+    try (Table stateTable = stateTableSupplier.get()) {
+      // Delete the regions
+      Scan scan = new Scan(makeTimeRegionKey(timeBytes, EMPTY_BYTE_ARRAY), REGION_TIME_KEY_PREFIX_STOP);
+      scan.addColumn(FAMILY, REGION_TIME_COL);
+      deleteFromScan(stateTable, scan);
+
+      // Delete the count
+      scan = new Scan(makeTimeRegionCountKey(timeBytes), REGION_TIME_COUNT_KEY_PREFIX_STOP);
+      scan.addColumn(FAMILY, REGION_TIME_COL);
+      deleteFromScan(stateTable, scan);
+    }
+  }
+
+  // ---------------------------------------------------------------------
+  // ------- Methods for inactive transaction bound for given time -------
+  // ---------------------------------------------------------------------
+  // Key: 0x3<inverted time>
+  // Col 'p': <inactive transaction bound>
+  // ---------------------------------------------------------------------
+
+  /**
+   * Persist inactive transaction bound for a given time. This is the smallest not in-progress transaction that
+   * will not have writes in any HBase regions that are created after the given time.
+   *
+   * @param time time in milliseconds
+   * @param inactiveTransactionBound inactive transaction bound for the given time
+   * @throws IOException when not able to persist the data to HBase
+   */
+  public void saveInactiveTransactionBoundForTime(long time, long inactiveTransactionBound) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      Put put = new Put(makeInactiveTransactionBoundTimeKey(Bytes.toBytes(getInvertedTime(time))));
+      put.addColumn(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL, Bytes.toBytes(inactiveTransactionBound));
+      stateTable.put(put);
+    }
+  }
+
+  /**
+   * Return inactive transaction bound for the given time.
+   *
+   * @param time time in milliseconds
+   * @return inactive transaction bound for the given time
+   * @throws IOException when not able to read the data from HBase
+   */
+  public long getInactiveTransactionBoundForTime(long time) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      Get get = new Get(makeInactiveTransactionBoundTimeKey(Bytes.toBytes(getInvertedTime(time))));
+      get.addColumn(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL);
+      byte[] result = stateTable.get(get).getValue(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL);
+      return result == null ? -1 : Bytes.toLong(result);
+    }
+  }
+
+  /**
+   * Delete all inactive transaction bounds recorded for a time less than the given time
+   *
+   * @param time time in milliseconds
+   * @throws IOException when not able to delete data in HBase
+   */
+  public void deleteInactiveTransactionBoundsOnOrBeforeTime(long time) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      Scan scan = new Scan(makeInactiveTransactionBoundTimeKey(Bytes.toBytes(getInvertedTime(time))),
+                           INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX_STOP);
+      scan.addColumn(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL);
+      deleteFromScan(stateTable, scan);
+    }
+  }
+
+  // --------------------------------------------------------
+  // ------- Methods for empty regions at a given time -------
+  // --------------------------------------------------------
+  // Key: 0x4<time><region-id>
+  // Col 'e': <empty byte array>
+  // --------------------------------------------------------
+
+  /**
+   * Save the given region as empty as of the given time.
+   *
+   * @param time time in milliseconds
+   * @param regionId region id
+   */
+  public void saveEmptyRegionForTime(long time, byte[] regionId) throws IOException {
+    byte[] timeBytes = Bytes.toBytes(time);
+    try (Table stateTable = stateTableSupplier.get()) {
+      Put put = new Put(makeEmptyRegionTimeKey(timeBytes, regionId));
+      put.addColumn(FAMILY, EMPTY_REGION_TIME_COL, COL_VAL);
+      stateTable.put(put);
+    }
+  }
+
+  /**
+   * Return regions that were recorded as empty after the given time.
+   *
+   * @param time time in milliseconds
+   * @param includeRegions If not null, the returned set will be an intersection of the includeRegions set
+   *                       and the empty regions after the given time
+   */
+  public SortedSet<byte[]> getEmptyRegionsAfterTime(long time, @Nullable SortedSet<byte[]> includeRegions)
+    throws IOException {
+    SortedSet<byte[]> emptyRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    try (Table stateTable = stateTableSupplier.get()) {
+      Scan scan = new Scan(makeEmptyRegionTimeKey(Bytes.toBytes(time + 1), EMPTY_BYTE_ARRAY),
+                           EMPTY_REGION_TIME_KEY_PREFIX_STOP);
+      scan.addColumn(FAMILY, EMPTY_REGION_TIME_COL);
+
+      try (ResultScanner scanner = stateTable.getScanner(scan)) {
+        Result next;
+        while ((next = scanner.next()) != null) {
+          byte[] emptyRegion = getEmptyRegionFromKey(next.getRow());
+          if (includeRegions == null || includeRegions.contains(emptyRegion)) {
+            emptyRegions.add(emptyRegion);
+          }
+        }
+      }
+    }
+    return Collections.unmodifiableSortedSet(emptyRegions);
+  }
+
+  /**
+   * Delete empty region records saved on or before the given time.
+   *
+   * @param time time in milliseconds
+   */
+  public void deleteEmptyRegionsOnOrBeforeTime(long time) throws IOException {
+    try (Table stateTable = stateTableSupplier.get()) {
+      Scan scan = new Scan();
+      scan.setStopRow(makeEmptyRegionTimeKey(Bytes.toBytes(time + 1), EMPTY_BYTE_ARRAY));
+      scan.addColumn(FAMILY, EMPTY_REGION_TIME_COL);
+      deleteFromScan(stateTable, scan);
+    }
+  }
+
+  @VisibleForTesting
+  void deleteFromScan(Table stateTable, Scan scan) throws IOException {
+    try (ResultScanner scanner = stateTable.getScanner(scan)) {
+      Result next;
+      while ((next = scanner.next()) != null) {
+        stateTable.delete(new Delete(next.getRow()));
+      }
+    }
+  }
+
+  private byte[] makeRegionKey(byte[] regionId) {
+    return Bytes.add(REGION_KEY_PREFIX, regionId);
+  }
+
+  private byte[] getRegionFromKey(byte[] regionKey) {
+    int prefixLen = REGION_KEY_PREFIX.length;
+    return Bytes.copy(regionKey, prefixLen, regionKey.length - prefixLen);
+  }
+
+  private byte[] makeTimeRegionKey(byte[] time, byte[] regionId) {
+    return Bytes.add(REGION_TIME_KEY_PREFIX, time, regionId);
+  }
+
+  private byte[] makeTimeRegionCountKey(byte[] time) {
+    return Bytes.add(REGION_TIME_COUNT_KEY_PREFIX, time);
+  }
+
+  private byte[] makeInactiveTransactionBoundTimeKey(byte[] time) {
+    return Bytes.add(INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX, time);
+  }
+
+  private Map.Entry<Long, byte[]> getTimeRegion(byte[] key) {
+    int offset = REGION_TIME_KEY_PREFIX.length;
+    long time = getInvertedTime(Bytes.toLong(key, offset));
+    offset += Bytes.SIZEOF_LONG;
+    byte[] regionName = Bytes.copy(key, offset, key.length - offset);
+    return Maps.immutableEntry(time, regionName);
+  }
+
+  private byte[] makeEmptyRegionTimeKey(byte[] time, byte[] regionId) {
+    return Bytes.add(EMPTY_REGION_TIME_KEY_PREFIX, time, regionId);
+  }
+
+  private byte[] getEmptyRegionFromKey(byte[] key) {
+    int prefixLen = EMPTY_REGION_TIME_KEY_PREFIX.length + Bytes.SIZEOF_LONG;
+    return Bytes.copy(key, prefixLen, key.length - prefixLen);
+  }
+
+  private long getInvertedTime(long time) {
+    return Long.MAX_VALUE - time;
+  }
+
+  /**
+   * Supplies table for persisting state
+   */
+  public interface TableSupplier {
+    Table get() throws IOException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java
new file mode 100644
index 0000000..84c480a
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java
@@ -0,0 +1,373 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+import org.apache.tephra.txprune.TransactionPruningPlugin;
+import org.apache.tephra.util.TxUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+/**
+ * Default implementation of the {@link TransactionPruningPlugin} for HBase.
+ *
+ * This plugin determines the prune upper bound for transactional HBase tables that use
+ * coprocessor {@link TransactionProcessor}.
+ *
+ * <h3>State storage:</h3>
+ *
+ * This plugin expects the TransactionProcessor to save the prune upper bound for invalid transactions
+ * after every major compaction of a region. Let's call this <i>(region, prune upper bound)</i>.
+ * In addition, the plugin also persists the following information on a run at time <i>t</i>
+ * <ul>
+ *   <li>
+ *     <i>(t, set of regions)</i>: Set of transactional regions at time <i>t</i>.
+ *     Transactional regions are regions of the tables that have the coprocessor TransactionProcessor
+ *     attached to them.
+ *   </li>
+ *   <li>
+ *     <i>(t, inactive transaction bound)</i>: This is the smallest not in-progress transaction that
+ *     will not have writes in any HBase regions that are created after time <i>t</i>.
+ *     This value is determined by the Transaction Service based on the transaction state at time <i>t</i>
+ *     and passed on to the plugin.
+ *   </li>
+ * </ul>
+ *
+ * <h3>Computing prune upper bound:</h3>
+ *
+ * In a typical HBase instance, there can be a constant change in the number of regions due to region creations,
+ * splits and merges. At any given time there can always be a region on which a major compaction has not been run.
+ * Since the prune upper bound will get recorded for a region only after a major compaction,
+ * using only the latest set of regions we may not be able to find the
+ * prune upper bounds for all the current regions. Hence we persist the set of regions that exist at that time
+ * of each run of the plugin, and use historical region set for time <i>t</i>, <i>t - 1</i>, etc.
+ * to determine the prune upper bound.
+ *
+ * From the regions saved at time <i>t</i>, <i>t - 1</i>, etc.,
+ * the plugin tries to find the latest <i>(t, set of regions)</i> where all regions have been major compacted,
+ * i.e, all regions have prune upper bound recorded in <i>(region, prune upper bound)</i>.
+ * <br/>
+ * If such a set is found for time <i>t1</i>, the prune upper bound returned by the plugin is the minimum of
+ * <ul>
+ *   <li>Prune upper bounds of regions in set <i>(t1, set of regions)</i></li>
+ *   <li>Inactive transaction bound from <i>(t1, inactive transaction bound)</i></li>
+ * </ul>
+ *
+ * <p/>
+ * Above, when we find <i>(t1, set of regions)</i>, there may a region that was created after time <i>t1</i>,
+ * but has a data write from an invalid transaction that is smaller than the prune upper bounds of all
+ * regions in <i>(t1, set of regions)</i>. This is possible because <i>(region, prune upper bound)</i> persisted by
+ * TransactionProcessor is always the latest prune upper bound for a region.
+ * <br/>
+ * However a region created after time <i>t1</i> cannot have writes from an invalid transaction that is smaller than
+ * inactive transaction bound at the time the region was created.
+ * Since we limit the plugin prune upper bound using <i>(t1, inactive transaction bound)</i>,
+ * there should be no invalid transactions smaller than the plugin prune upper bound with writes in any
+ * transactional region of this HBase instance.
+ *
+ * <p/>
+ * Note: If your tables uses a transactional coprocessor other than TransactionProcessor,
+ * then you may need to write a new plugin to compute prune upper bound for those tables.
+ */
+@SuppressWarnings("WeakerAccess")
+public class HBaseTransactionPruningPlugin implements TransactionPruningPlugin {
+  public static final Logger LOG = LoggerFactory.getLogger(HBaseTransactionPruningPlugin.class);
+
+  protected Configuration conf;
+  protected Connection connection;
+  protected DataJanitorState dataJanitorState;
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+    this.conf = conf;
+    this.connection = ConnectionFactory.createConnection(conf);
+
+    final TableName stateTable = TableName.valueOf(conf.get(TxConstants.TransactionPruning.PRUNE_STATE_TABLE,
+                                                            TxConstants.TransactionPruning.DEFAULT_PRUNE_STATE_TABLE));
+    LOG.info("Initializing plugin with state table {}", stateTable.getNameWithNamespaceInclAsString());
+    createPruneTable(stateTable);
+    this.dataJanitorState = new DataJanitorState(new DataJanitorState.TableSupplier() {
+      @Override
+      public Table get() throws IOException {
+        return connection.getTable(stateTable);
+      }
+    });
+  }
+
+  /**
+   * Determines prune upper bound for the data store as mentioned above.
+   */
+  @Override
+  public long fetchPruneUpperBound(long time, long inactiveTransactionBound) throws IOException {
+    LOG.debug("Fetching prune upper bound for time {} and inactive transaction bound {}",
+              time, inactiveTransactionBound);
+    if (time < 0 || inactiveTransactionBound < 0) {
+      return -1;
+    }
+
+    // Get all the current transactional regions
+    SortedSet<byte[]> transactionalRegions = getTransactionalRegions();
+    if (!transactionalRegions.isEmpty()) {
+      LOG.debug("Saving {} transactional regions for time {}", transactionalRegions.size(), time);
+      dataJanitorState.saveRegionsForTime(time, transactionalRegions);
+      // Save inactive transaction bound for time as the final step.
+      // We can then use its existence to make sure that the data for a given time is complete or not
+      LOG.debug("Saving inactive transaction bound {} for time {}", inactiveTransactionBound, time);
+      dataJanitorState.saveInactiveTransactionBoundForTime(time, inactiveTransactionBound);
+    }
+
+    return computePruneUpperBound(new TimeRegions(time, transactionalRegions));
+  }
+
+  /**
+   * After invalid list has been pruned, this cleans up state information that is no longer required.
+   * This includes -
+   * <ul>
+   *   <li>
+   *     <i>(region, prune upper bound)</i> - prune upper bound for regions that are older
+   *     than maxPrunedInvalid
+   *   </li>
+   *   <li>
+   *     <i>(t, set of regions) - Regions set that were recorded on or before the start time
+   *     of maxPrunedInvalid
+   *   </li>
+   *   <li>
+   *     (t, inactive transaction bound) - Smallest not in-progress transaction without any writes in new regions
+   *     information recorded on or before the start time of maxPrunedInvalid
+   *   </li>
+   * </ul>
+   */
+  @Override
+  public void pruneComplete(long time, long maxPrunedInvalid) throws IOException {
+    LOG.debug("Prune complete for time {} and prune upper bound {}", time, maxPrunedInvalid);
+    if (time < 0 || maxPrunedInvalid < 0) {
+      return;
+    }
+
+    // Get regions for the current time, so as to not delete the prune upper bounds for them.
+    // The prune upper bounds for regions are recorded by TransactionProcessor and the deletion
+    // is done by this class. To avoid update/delete race condition, we only delete prune upper
+    // bounds for the stale regions.
+    TimeRegions regionsToExclude = dataJanitorState.getRegionsOnOrBeforeTime(time);
+    if (regionsToExclude != null) {
+      LOG.debug("Deleting prune upper bounds smaller than {} for stale regions", maxPrunedInvalid);
+      dataJanitorState.deletePruneUpperBounds(maxPrunedInvalid, regionsToExclude.getRegions());
+    } else {
+      LOG.warn("Cannot find saved regions on or before time {}", time);
+    }
+    long pruneTime = TxUtils.getTimestamp(maxPrunedInvalid);
+    LOG.debug("Deleting regions recorded before time {}", pruneTime);
+    dataJanitorState.deleteAllRegionsOnOrBeforeTime(pruneTime);
+    LOG.debug("Deleting inactive transaction bounds recorded on or before time {}", pruneTime);
+    dataJanitorState.deleteInactiveTransactionBoundsOnOrBeforeTime(pruneTime);
+    LOG.debug("Deleting empty regions recorded on or before time {}", pruneTime);
+    dataJanitorState.deleteEmptyRegionsOnOrBeforeTime(pruneTime);
+  }
+
+  @Override
+  public void destroy() {
+    LOG.info("Stopping plugin...");
+    try {
+      connection.close();
+    } catch (IOException e) {
+      LOG.error("Got exception while closing HBase connection", e);
+    }
+  }
+
+  /**
+   * Create the prune state table given the {@link TableName} if the table doesn't exist already.
+   *
+   * @param stateTable prune state table name
+   */
+  protected void createPruneTable(TableName stateTable) throws IOException {
+    try (Admin admin = this.connection.getAdmin()) {
+      if (admin.tableExists(stateTable)) {
+        LOG.debug("Not creating pruneStateTable {} since it already exists.",
+                  stateTable.getNameWithNamespaceInclAsString());
+        return;
+      }
+
+      HTableDescriptor htd = new HTableDescriptor(stateTable);
+      htd.addFamily(new HColumnDescriptor(DataJanitorState.FAMILY).setMaxVersions(1));
+      admin.createTable(htd);
+      LOG.info("Created pruneTable {}", stateTable.getNameWithNamespaceInclAsString());
+    } catch (TableExistsException ex) {
+      // Expected if the prune state table is being created at the same time by another client
+      LOG.debug("Not creating pruneStateTable {} since it already exists.",
+                stateTable.getNameWithNamespaceInclAsString(), ex);
+    }
+  }
+
+  /**
+   * Returns whether the table is a transactional table. By default, it is a table is identified as a transactional
+   * table if it has a the coprocessor {@link TransactionProcessor} attached to it. Should be overriden if the users
+   * attach a different coprocessor.
+   *
+   * @param tableDescriptor {@link HTableDescriptor} of the table
+   * @return true if the table is transactional
+   */
+  protected boolean isTransactionalTable(HTableDescriptor tableDescriptor) {
+    return tableDescriptor.hasCoprocessor(TransactionProcessor.class.getName());
+  }
+
+  protected SortedSet<byte[]> getTransactionalRegions() throws IOException {
+    SortedSet<byte[]> regions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    try (Admin admin = connection.getAdmin()) {
+      HTableDescriptor[] tableDescriptors = admin.listTables();
+      LOG.debug("Got {} tables to process", tableDescriptors == null ? 0 : tableDescriptors.length);
+      if (tableDescriptors != null) {
+        for (HTableDescriptor tableDescriptor : tableDescriptors) {
+          if (isTransactionalTable(tableDescriptor)) {
+            List<HRegionInfo> tableRegions = admin.getTableRegions(tableDescriptor.getTableName());
+            LOG.debug("Regions for table {}: {}", tableDescriptor.getTableName(), tableRegions);
+            if (tableRegions != null) {
+              for (HRegionInfo region : tableRegions) {
+                regions.add(region.getRegionName());
+              }
+            }
+          } else {
+            LOG.debug("{} is not a transactional table", tableDescriptor.getTableName());
+          }
+        }
+      }
+    }
+    return regions;
+  }
+
+  /**
+   * Try to find the latest set of regions in which all regions have been major compacted, and
+   * compute prune upper bound from them. Starting from newest to oldest, this looks into the
+   * region set that has been saved periodically, and joins it with the prune upper bound data
+   * for a region recorded after a major compaction.
+   *
+   * @param timeRegions the latest set of regions
+   * @return prune upper bound
+   * @throws IOException when not able to talk to HBase
+   */
+  private long computePruneUpperBound(TimeRegions timeRegions) throws IOException {
+    do {
+      LOG.debug("Computing prune upper bound for {}", timeRegions);
+      SortedSet<byte[]> transactionalRegions = timeRegions.getRegions();
+      long time = timeRegions.getTime();
+
+      long inactiveTransactionBound = dataJanitorState.getInactiveTransactionBoundForTime(time);
+      LOG.debug("Got inactive transaction bound {}", inactiveTransactionBound);
+      // If inactiveTransactionBound is not recorded then that means the data is not complete for these regions
+      if (inactiveTransactionBound == -1) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Ignoring regions for time {} as no inactiveTransactionBound was found for that time, " +
+                      "and hence the data must be incomplete", time);
+        }
+        continue;
+      }
+
+      // Get the prune upper bounds for all the transactional regions
+      Map<byte[], Long> pruneUpperBoundRegions =
+        dataJanitorState.getPruneUpperBoundForRegions(transactionalRegions);
+      logPruneUpperBoundRegions(pruneUpperBoundRegions);
+
+      // Use inactiveTransactionBound as the prune upper bound for the empty regions since the regions that are
+      // recorded as empty after inactiveTransactionBoundTime will not have invalid data
+      // for transactions started on or before inactiveTransactionBoundTime
+      pruneUpperBoundRegions = handleEmptyRegions(inactiveTransactionBound, transactionalRegions,
+                                                  pruneUpperBoundRegions);
+
+      // If prune upper bounds are found for all the transactional regions, then compute the prune upper bound
+      // across all regions
+      if (!transactionalRegions.isEmpty() &&
+        pruneUpperBoundRegions.size() == transactionalRegions.size()) {
+        Long minPruneUpperBoundRegions = Collections.min(pruneUpperBoundRegions.values());
+        long pruneUpperBound = Math.min(inactiveTransactionBound, minPruneUpperBoundRegions);
+        LOG.debug("Found prune upper bound {} for time {}", pruneUpperBound, time);
+        return pruneUpperBound;
+      } else {
+        if (LOG.isDebugEnabled()) {
+          Sets.SetView<byte[]> difference =
+            Sets.difference(transactionalRegions, pruneUpperBoundRegions.keySet());
+          LOG.debug("Ignoring regions for time {} because the following regions did not record a pruneUpperBound: {}",
+                    time, Iterables.transform(difference, TimeRegions.BYTE_ARR_TO_STRING_FN));
+        }
+      }
+
+      timeRegions = dataJanitorState.getRegionsOnOrBeforeTime(time - 1);
+    } while (timeRegions != null);
+    return -1;
+  }
+
+  private Map<byte[], Long> handleEmptyRegions(long inactiveTransactionBound,
+                                               SortedSet<byte[]> transactionalRegions,
+                                               Map<byte[], Long> pruneUpperBoundRegions) throws IOException {
+    long inactiveTransactionBoundTime = TxUtils.getTimestamp(inactiveTransactionBound);
+    SortedSet<byte[]> emptyRegions =
+      dataJanitorState.getEmptyRegionsAfterTime(inactiveTransactionBoundTime, transactionalRegions);
+    LOG.debug("Got empty transactional regions for inactive transaction bound time {}: {}",
+              inactiveTransactionBoundTime, Iterables.transform(emptyRegions, TimeRegions.BYTE_ARR_TO_STRING_FN));
+
+    // The regions that are recorded as empty after inactiveTransactionBoundTime will not have invalid data
+    // for transactions started before or on inactiveTransactionBoundTime. Hence we can consider the prune upper bound
+    // for these empty regions as inactiveTransactionBound
+    Map<byte[], Long> pubWithEmptyRegions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    pubWithEmptyRegions.putAll(pruneUpperBoundRegions);
+    for (byte[] emptyRegion : emptyRegions) {
+      if (!pruneUpperBoundRegions.containsKey(emptyRegion)) {
+        pubWithEmptyRegions.put(emptyRegion, inactiveTransactionBound);
+      }
+    }
+    return Collections.unmodifiableMap(pubWithEmptyRegions);
+  }
+
+  private void logPruneUpperBoundRegions(Map<byte[], Long> pruneUpperBoundRegions) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Got region - prune upper bound map: {}",
+                Iterables.transform(pruneUpperBoundRegions.entrySet(),
+                                    new Function<Map.Entry<byte[], Long>, Map.Entry<String, Long>>() {
+                                      @Override
+                                      public Map.Entry<String, Long> apply(Map.Entry<byte[], Long> input) {
+                                        String regionName = TimeRegions.BYTE_ARR_TO_STRING_FN.apply(input.getKey());
+                                        return Maps.immutableEntry(regionName, input.getValue());
+                                      }
+                                    }));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/InvalidListPruningDebug.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/InvalidListPruningDebug.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/InvalidListPruningDebug.java
new file mode 100644
index 0000000..443c998
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/InvalidListPruningDebug.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.MinMaxPriorityQueue;
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.txprune.RegionPruneInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import javax.annotation.Nullable;
+
+/**
+ * Invalid List Pruning Debug Tool.
+ */
+public class InvalidListPruningDebug {
+  private static final Logger LOG = LoggerFactory.getLogger(InvalidListPruningDebug.class);
+  private static final Gson GSON = new Gson();
+  private DataJanitorState dataJanitorState;
+  private Connection connection;
+  private TableName tableName;
+
+  /**
+   * Initialize the Invalid List Debug Tool.
+   * @param conf {@link Configuration}
+   * @throws IOException
+   */
+  public void initialize(final Configuration conf) throws IOException {
+    LOG.debug("InvalidListPruningDebugMain : initialize method called");
+    connection = ConnectionFactory.createConnection(conf);
+    tableName = TableName.valueOf(conf.get(TxConstants.TransactionPruning.PRUNE_STATE_TABLE,
+                                           TxConstants.TransactionPruning.DEFAULT_PRUNE_STATE_TABLE));
+    dataJanitorState = new DataJanitorState(new DataJanitorState.TableSupplier() {
+      @Override
+      public Table get() throws IOException {
+        return connection.getTable(tableName);
+      }
+    });
+  }
+
+  public void destroy() throws IOException {
+    if (connection != null) {
+      connection.close();
+    }
+  }
+
+  /**
+   * Returns a set of regions that are live but are not empty nor have a prune upper bound recorded. These regions
+   * will stop the progress of pruning.
+   *
+   * @param numRegions number of regions
+   * @return {@link Set} of regions that needs to be compacted and flushed
+   */
+  public Set<String> getRegionsToBeCompacted(Integer numRegions) throws IOException {
+    // Fetch the live regions
+    Map<Long, SortedSet<String>> latestTimeRegion = getRegionsOnOrBeforeTime(System.currentTimeMillis());
+    if (latestTimeRegion.isEmpty()) {
+      return new HashSet<>();
+    }
+
+    Long timestamp = latestTimeRegion.keySet().iterator().next();
+    SortedSet<String> liveRegions = latestTimeRegion.get(timestamp);
+
+    SortedSet<byte[]> emptyRegions = dataJanitorState.getEmptyRegionsAfterTime(timestamp, null);
+    SortedSet<String> emptyRegionNames = new TreeSet<>();
+    Iterable<String> regionStrings = Iterables.transform(emptyRegions, TimeRegions.BYTE_ARR_TO_STRING_FN);
+    for (String regionString : regionStrings) {
+      emptyRegionNames.add(regionString);
+    }
+
+    Set<String> nonEmptyRegions = Sets.newHashSet(Sets.difference(liveRegions, emptyRegionNames));
+
+    // Get all pruned regions and remove them from the nonEmptyRegions, resulting in a set of regions that are
+    // not empty and have not been registered prune upper bound
+    Queue<RegionPruneInfo> prunedRegions = getIdleRegions(-1);
+    for (RegionPruneInfo prunedRegion : prunedRegions) {
+      if (nonEmptyRegions.contains(prunedRegion.getRegionNameAsString())) {
+        nonEmptyRegions.remove(prunedRegion.getRegionNameAsString());
+      }
+    }
+
+    if ((numRegions < 0) || (numRegions >= nonEmptyRegions.size())) {
+      return nonEmptyRegions;
+    }
+
+    Set<String> subsetRegions = new HashSet<>(numRegions);
+    for (String regionName : nonEmptyRegions) {
+      if (subsetRegions.size() == numRegions) {
+        break;
+      }
+      subsetRegions.add(regionName);
+    }
+    return subsetRegions;
+  }
+
+  /**
+   * Return a list of RegionPruneInfo. These regions are the ones that have the lowest prune upper bounds.
+   * If -1 is passed in, all the regions and their prune upper bound will be returned. Note that only the regions
+   * that are known to be live will be returned.
+   *
+   * @param numRegions number of regions
+   * @return Map of region name and its prune upper bound
+   */
+  public Queue<RegionPruneInfo> getIdleRegions(Integer numRegions) throws IOException {
+    List<RegionPruneInfo> regionPruneInfos = dataJanitorState.getPruneInfoForRegions(null);
+    if (regionPruneInfos.isEmpty()) {
+      return new LinkedList<>();
+    }
+
+    // Create a set with region names
+    Set<String> pruneRegionNameSet = new HashSet<>();
+    for (RegionPruneInfo regionPruneInfo : regionPruneInfos) {
+      pruneRegionNameSet.add(regionPruneInfo.getRegionNameAsString());
+    }
+
+    // Fetch the live regions
+    Map<Long, SortedSet<String>> latestTimeRegion = getRegionsOnOrBeforeTime(System.currentTimeMillis());
+    if (!latestTimeRegion.isEmpty()) {
+      SortedSet<String> liveRegions = latestTimeRegion.values().iterator().next();
+      Set<String> liveRegionsWithPruneInfo = Sets.intersection(liveRegions, pruneRegionNameSet);
+      List<RegionPruneInfo> liveRegionWithPruneInfoList = new ArrayList<>();
+      for (RegionPruneInfo regionPruneInfo : regionPruneInfos) {
+        if (liveRegionsWithPruneInfo.contains(regionPruneInfo.getRegionNameAsString())) {
+          liveRegionWithPruneInfoList.add(regionPruneInfo);
+        }
+      }
+
+      // Use the subset of live regions and prune regions
+      regionPruneInfos = liveRegionWithPruneInfoList;
+    }
+
+    if (numRegions < 0) {
+      numRegions = regionPruneInfos.size();
+    }
+
+    Queue<RegionPruneInfo> lowestPrunes = MinMaxPriorityQueue.orderedBy(new Comparator<RegionPruneInfo>() {
+      @Override
+      public int compare(RegionPruneInfo o1, RegionPruneInfo o2) {
+        return (int) (o1.getPruneUpperBound() - o2.getPruneUpperBound());
+      }
+    }).maximumSize(numRegions).create();
+
+    for (RegionPruneInfo pruneInfo : regionPruneInfos) {
+      lowestPrunes.add(pruneInfo);
+    }
+    return lowestPrunes;
+  }
+
+  /**
+   * Return the prune upper bound value of a given region. If no prune upper bound has been written for this region yet,
+   * it will return a null.
+   *
+   * @param regionId region id
+   * @return {@link RegionPruneInfo} of the region
+   * @throws IOException if there are any errors while trying to fetch the {@link RegionPruneInfo}
+   */
+  @Nullable
+  public RegionPruneInfo getRegionPruneInfo(String regionId) throws IOException {
+    return dataJanitorState.getPruneInfoForRegion(Bytes.toBytesBinary(regionId));
+  }
+
+  /**
+   *
+   * @param time Given a time, provide the {@link TimeRegions} at or before that time
+   * @return transactional regions that are present at or before the given time
+   * @throws IOException if there are any errors while trying to fetch the {@link TimeRegions}
+   */
+  public Map<Long, SortedSet<String>> getRegionsOnOrBeforeTime(Long time) throws IOException {
+    Map<Long, SortedSet<String>> regionMap = new HashMap<>();
+    TimeRegions timeRegions = dataJanitorState.getRegionsOnOrBeforeTime(time);
+    if (timeRegions == null) {
+      return regionMap;
+    }
+    SortedSet<String> regionNames = new TreeSet<>();
+    Iterable<String> regionStrings = Iterables.transform(timeRegions.getRegions(), TimeRegions.BYTE_ARR_TO_STRING_FN);
+    for (String regionString : regionStrings) {
+      regionNames.add(regionString);
+    }
+    regionMap.put(timeRegions.getTime(), regionNames);
+    return regionMap;
+  }
+
+  private void printUsage(PrintWriter pw) {
+    pw.println("Usage : org.apache.tephra.hbase.txprune.InvalidListPruning <command> <parameter>");
+    pw.println("Available commands, corresponding parameters are:");
+    pw.println("****************************************************");
+    pw.println("time-region ts");
+    pw.println("Desc: Prints out the transactional regions present in HBase at time 'ts' (in milliseconds) " +
+                 "or the latest time before time 'ts'.");
+    pw.println("idle-regions limit");
+    pw.println("Desc: Prints out 'limit' number of regions which has the lowest prune upper bounds. If '-1' is " +
+                 "provided as the limit, prune upper bounds of all regions are returned.");
+    pw.println("prune-info region-name-as-string");
+    pw.println("Desc: Prints out the Pruning information for the region 'region-name-as-string'");
+    pw.println("to-compact-regions limit");
+    pw.println("Desc: Prints out 'limit' number of regions that are active, but are not empty, " +
+                 "and have not registered a prune upper bound.");
+  }
+
+  private boolean execute(String[] args) throws IOException {
+    try (PrintWriter pw = new PrintWriter(System.out)) {
+      if (args.length != 2) {
+        printUsage(pw);
+        return false;
+      }
+
+      String command = args[0];
+      String parameter = args[1];
+      if ("time-region".equals(command)) {
+        Long time = Long.parseLong(parameter);
+        Map<Long, SortedSet<String>> timeRegion = getRegionsOnOrBeforeTime(time);
+        pw.println(GSON.toJson(timeRegion));
+        return true;
+      } else if ("idle-regions".equals(command)) {
+        Integer numRegions = Integer.parseInt(parameter);
+        Queue<RegionPruneInfo> regionPruneInfos = getIdleRegions(numRegions);
+        pw.println(GSON.toJson(regionPruneInfos));
+        return true;
+      } else if ("prune-info".equals(command)) {
+        RegionPruneInfo regionPruneInfo = getRegionPruneInfo(parameter);
+        if (regionPruneInfo != null) {
+          pw.println(GSON.toJson(regionPruneInfo));
+        } else {
+          pw.println(String.format("No prune info found for the region %s.", parameter));
+        }
+        return true;
+      } else if ("to-compact-regions".equals(command)) {
+        Integer numRegions = Integer.parseInt(parameter);
+        Set<String> toBeCompactedRegions = getRegionsToBeCompacted(numRegions);
+        pw.println(GSON.toJson(toBeCompactedRegions));
+        return true;
+      } else {
+        pw.println(String.format("%s is not a valid command.", command));
+        printUsage(pw);
+        return false;
+      }
+    }
+  }
+
+  public static void main(String[] args) {
+    Configuration hConf = HBaseConfiguration.create();
+    InvalidListPruningDebug pruningDebug = new InvalidListPruningDebug();
+    try {
+      pruningDebug.initialize(hConf);
+      boolean success = pruningDebug.execute(args);
+      pruningDebug.destroy();
+      if (!success) {
+        System.exit(1);
+      }
+    } catch (IOException ex) {
+      LOG.error("Received an exception while trying to execute the debug tool. ", ex);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriter.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriter.java
new file mode 100644
index 0000000..677710b
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriter.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.txprune;
+
+import com.google.common.util.concurrent.AbstractIdleService;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Thread that will write the the prune upper bound. An instance of this class should be obtained only
+ * through {@link PruneUpperBoundWriterSupplier} which will also handle the lifecycle of this instance.
+ */
+public class PruneUpperBoundWriter extends AbstractIdleService {
+  private static final Log LOG = LogFactory.getLog(PruneUpperBoundWriter.class);
+
+  private final TableName tableName;
+  private final DataJanitorState dataJanitorState;
+  private final long pruneFlushInterval;
+  // Map of region name -> prune upper bound
+  private final ConcurrentSkipListMap<byte[], Long> pruneEntries;
+  // Map of region name -> time the region was found to be empty
+  private final ConcurrentSkipListMap<byte[], Long> emptyRegions;
+
+  private volatile Thread flushThread;
+  private volatile boolean stopped;
+
+  private long lastChecked;
+
+  @SuppressWarnings("WeakerAccess")
+  public PruneUpperBoundWriter(TableName tableName, DataJanitorState dataJanitorState, long pruneFlushInterval) {
+    this.tableName = tableName;
+    this.dataJanitorState = dataJanitorState;
+    this.pruneFlushInterval = pruneFlushInterval;
+    this.pruneEntries = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
+    this.emptyRegions = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public void persistPruneEntry(byte[] regionName, long pruneUpperBound) {
+    warnIfNotRunning(regionName);
+    // The number of entries in this map is bound by the number of regions in this region server and thus it will not
+    // grow indefinitely
+    pruneEntries.put(regionName, pruneUpperBound);
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public void persistRegionEmpty(byte[] regionName, long time) {
+    warnIfNotRunning(regionName);
+    // The number of entries in this map is bound by the number of regions in this region server and thus it will not
+    // grow indefinitely
+    emptyRegions.put(regionName, time);
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public boolean isAlive() {
+    return flushThread != null && flushThread.isAlive();
+  }
+
+  @Override
+  protected void startUp() throws Exception {
+    LOG.info("Starting PruneUpperBoundWriter Thread.");
+    startFlushThread();
+  }
+
+  @Override
+  protected void shutDown() throws Exception {
+    LOG.info("Stopping PruneUpperBoundWriter Thread.");
+    stopped = true;
+    if (flushThread != null) {
+      flushThread.interrupt();
+      flushThread.join(TimeUnit.SECONDS.toMillis(1));
+      if (flushThread.isAlive()) {
+        flushThread.interrupt();
+        flushThread.join(TimeUnit.SECONDS.toMillis(1));
+      }
+    }
+  }
+
+  private void startFlushThread() {
+    flushThread = new Thread("tephra-prune-upper-bound-writer") {
+      @Override
+      public void run() {
+        while ((!isInterrupted()) && (!stopped)) {
+          long now = System.currentTimeMillis();
+          if (now > (lastChecked + pruneFlushInterval)) {
+            // should flush data
+            try {
+              User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                  // Record prune upper bound
+                  while (!pruneEntries.isEmpty()) {
+                    Map.Entry<byte[], Long> firstEntry = pruneEntries.firstEntry();
+                    dataJanitorState.savePruneUpperBoundForRegion(firstEntry.getKey(), firstEntry.getValue());
+                    // We can now remove the entry only if the key and value match with what we wrote since it is
+                    // possible that a new pruneUpperBound for the same key has been added
+                    pruneEntries.remove(firstEntry.getKey(), firstEntry.getValue());
+                  }
+                  // Record empty regions
+                  while (!emptyRegions.isEmpty()) {
+                    Map.Entry<byte[], Long> firstEntry = emptyRegions.firstEntry();
+                    dataJanitorState.saveEmptyRegionForTime(firstEntry.getValue(), firstEntry.getKey());
+                    // We can now remove the entry only if the key and value match with what we wrote since it is
+                    // possible that a new value for the same key has been added
+                    emptyRegions.remove(firstEntry.getKey(), firstEntry.getValue());
+                  }
+                  return null;
+                }
+              });
+            } catch (IOException ex) {
+              LOG.warn("Cannot record prune upper bound for a region to table " +
+                         tableName.getNameWithNamespaceInclAsString(), ex);
+            }
+            lastChecked = now;
+          }
+
+          try {
+            TimeUnit.SECONDS.sleep(1);
+          } catch (InterruptedException ex) {
+            interrupt();
+            break;
+          }
+        }
+
+        LOG.info("PruneUpperBound Writer thread terminated.");
+      }
+    };
+
+    flushThread.setDaemon(true);
+    flushThread.start();
+  }
+
+  private void warnIfNotRunning(byte[] regionName) {
+    if (!isRunning() || !isAlive()) {
+      LOG.warn(String.format("Trying to persist prune upper bound for region %s when writer is not %s!",
+                             Bytes.toStringBinary(regionName), isRunning() ? "alive" : "running"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplier.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplier.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplier.java
new file mode 100644
index 0000000..cb93fab
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplier.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase.txprune;
+
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.tephra.coprocessor.CacheSupplier;
+import org.apache.tephra.coprocessor.ReferenceCountedSupplier;
+
+/**
+ * Supplies instances of {@link PruneUpperBoundWriter} implementations.
+ */
+public class PruneUpperBoundWriterSupplier implements CacheSupplier<PruneUpperBoundWriter> {
+
+  private static final ReferenceCountedSupplier<PruneUpperBoundWriter> referenceCountedSupplier =
+    new ReferenceCountedSupplier<>(PruneUpperBoundWriter.class.getSimpleName());
+
+  private final Supplier<PruneUpperBoundWriter> supplier;
+
+  public PruneUpperBoundWriterSupplier(final TableName tableName, final DataJanitorState dataJanitorState,
+                                       final long pruneFlushInterval) {
+    this.supplier = new Supplier<PruneUpperBoundWriter>() {
+      @Override
+      public PruneUpperBoundWriter get() {
+        return new PruneUpperBoundWriter(tableName, dataJanitorState, pruneFlushInterval);
+      }
+    };
+  }
+
+  @Override
+  public PruneUpperBoundWriter get() {
+    return referenceCountedSupplier.getOrCreate(supplier);
+  }
+
+  public void release() {
+    referenceCountedSupplier.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/TimeRegions.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/TimeRegions.java b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/TimeRegions.java
new file mode 100644
index 0000000..4ac8887
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/main/java/org/apache/tephra/hbase/txprune/TimeRegions.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase.txprune;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.Objects;
+import java.util.SortedSet;
+
+/**
+ * Contains information on the set of transactional regions recorded at a given time
+ */
+@SuppressWarnings("WeakerAccess")
+public class TimeRegions {
+  static final Function<byte[], String> BYTE_ARR_TO_STRING_FN =
+    new Function<byte[], String>() {
+      @Override
+      public String apply(byte[] input) {
+        return Bytes.toStringBinary(input);
+      }
+    };
+
+  private final long time;
+  private final SortedSet<byte[]> regions;
+
+  public TimeRegions(long time, SortedSet<byte[]> regions) {
+    this.time = time;
+    this.regions = regions;
+  }
+
+  public long getTime() {
+    return time;
+  }
+
+  public SortedSet<byte[]> getRegions() {
+    return regions;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TimeRegions that = (TimeRegions) o;
+    return time == that.time &&
+      Objects.equals(regions, that.regions);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(time, regions);
+  }
+
+  @Override
+  public String toString() {
+    Iterable<String> regionStrings = Iterables.transform(regions, BYTE_ARR_TO_STRING_FN);
+    return "TimeRegions{" +
+      "time=" + time +
+      ", regions=[" + Joiner.on(" ").join(regionStrings) + "]" +
+      '}';
+  }
+}


[3/5] incubator-tephra git commit: Support for HBase 1.3.x

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/AbstractHBaseTableTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/AbstractHBaseTableTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/AbstractHBaseTableTest.java
new file mode 100644
index 0000000..560b0fe
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/AbstractHBaseTableTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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 org.apache.tephra.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Base class for tests that need a HBase cluster
+ */
+@SuppressWarnings("WeakerAccess")
+public abstract class AbstractHBaseTableTest {
+  protected static HBaseTestingUtility testUtil;
+  protected static HBaseAdmin hBaseAdmin;
+  protected static Configuration conf;
+
+  @BeforeClass
+  public static void startMiniCluster() throws Exception {
+    testUtil = conf == null ? new HBaseTestingUtility() : new HBaseTestingUtility(conf);
+    conf = testUtil.getConfiguration();
+
+    // Tune down the connection thread pool size
+    conf.setInt("hbase.hconnection.threads.core", 5);
+    conf.setInt("hbase.hconnection.threads.max", 10);
+    // Tunn down handler threads in regionserver
+    conf.setInt("hbase.regionserver.handler.count", 10);
+
+    // Set to random port
+    conf.setInt("hbase.master.port", 0);
+    conf.setInt("hbase.master.info.port", 0);
+    conf.setInt("hbase.regionserver.port", 0);
+    conf.setInt("hbase.regionserver.info.port", 0);
+
+    testUtil.startMiniCluster();
+    hBaseAdmin = testUtil.getHBaseAdmin();
+  }
+
+  @AfterClass
+  public static void shutdownMiniCluster() throws Exception {
+    try {
+      if (hBaseAdmin != null) {
+        hBaseAdmin.close();
+      }
+    } finally {
+      testUtil.shutdownMiniCluster();
+    }
+  }
+
+  protected static HTable createTable(byte[] tableName, byte[][] columnFamilies) throws Exception {
+    return createTable(tableName, columnFamilies, false,
+                       Collections.singletonList(TransactionProcessor.class.getName()));
+  }
+
+  protected static HTable createTable(byte[] tableName, byte[][] columnFamilies, boolean existingData,
+                                      List<String> coprocessors) throws Exception {
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+    for (byte[] family : columnFamilies) {
+      HColumnDescriptor columnDesc = new HColumnDescriptor(family);
+      columnDesc.setMaxVersions(Integer.MAX_VALUE);
+      columnDesc.setValue(TxConstants.PROPERTY_TTL, String.valueOf(100000)); // in millis
+      desc.addFamily(columnDesc);
+    }
+    if (existingData) {
+      desc.setValue(TxConstants.READ_NON_TX_DATA, "true");
+    }
+    // Divide individually to prevent any overflow
+    int priority = Coprocessor.PRIORITY_USER;
+    // order in list is the same order that coprocessors will be invoked
+    for (String coprocessor : coprocessors) {
+      desc.addCoprocessor(coprocessor, null, ++priority, null);
+    }
+    hBaseAdmin.createTable(desc);
+    testUtil.waitTableAvailable(tableName, 5000);
+    return new HTable(testUtil.getConfiguration(), tableName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/HBase13ConfigurationProviderTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/HBase13ConfigurationProviderTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/HBase13ConfigurationProviderTest.java
new file mode 100644
index 0000000..4c15632
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/HBase13ConfigurationProviderTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.tephra.util.AbstractConfigurationProviderTest;
+import org.apache.tephra.util.HBaseVersion;
+
+import java.util.Collection;
+
+/**
+ * Test for HBase 1.3 version specific behavior.
+ */
+public class HBase13ConfigurationProviderTest extends AbstractConfigurationProviderTest {
+  @Override
+  protected Collection<HBaseVersion.Version> getExpectedVersions() {
+    return ImmutableList.of(HBaseVersion.Version.HBASE_13);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/8f958edb/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
new file mode 100644
index 0000000..11ffd1a
--- /dev/null
+++ b/tephra-hbase-compat-1.3/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
@@ -0,0 +1,1726 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      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 org.apache.tephra.hbase;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+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.OperationWithAttributes;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.LongComparator;
+import org.apache.hadoop.hbase.filter.ValueFilter;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.tephra.Transaction;
+import org.apache.tephra.TransactionConflictException;
+import org.apache.tephra.TransactionContext;
+import org.apache.tephra.TransactionManager;
+import org.apache.tephra.TransactionSystemClient;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.TxConstants.ConflictDetection;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+import org.apache.tephra.inmemory.InMemoryTxSystemClient;
+import org.apache.tephra.metrics.TxMetricsCollector;
+import org.apache.tephra.persist.HDFSTransactionStateStorage;
+import org.apache.tephra.persist.InMemoryTransactionStateStorage;
+import org.apache.tephra.persist.TransactionStateStorage;
+import org.apache.tephra.snapshot.SnapshotCodecProvider;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for TransactionAwareHTables.
+ */
+public class TransactionAwareHTableTest extends AbstractHBaseTableTest {
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionAwareHTableTest.class);
+
+  static TransactionStateStorage txStateStorage;
+  static TransactionManager txManager;
+  private TransactionContext transactionContext;
+  private TransactionAwareHTable transactionAwareHTable;
+  private HTable hTable;
+  
+  @ClassRule
+  public static TemporaryFolder tmpFolder = new TemporaryFolder();
+  
+  private static MiniDFSCluster dfsCluster;
+  
+  public static void tearDownAfterClass() throws Exception {
+    dfsCluster.shutdown();
+  }
+
+  private static final class TestBytes {
+    private static final byte[] table = Bytes.toBytes("testtable");
+    private static final byte[] family = Bytes.toBytes("f1");
+    private static final byte[] family2 = Bytes.toBytes("f2");
+    private static final byte[] qualifier = Bytes.toBytes("col1");
+    private static final byte[] qualifier2 = Bytes.toBytes("col2");
+    private static final byte[] row = Bytes.toBytes("row");
+    private static final byte[] row2 = Bytes.toBytes("row2");
+    private static final byte[] row3 = Bytes.toBytes("row3");
+    private static final byte[] row4 = Bytes.toBytes("row4");
+    private static final byte[] value = Bytes.toBytes("value");
+    private static final byte[] value2 = Bytes.toBytes("value2");
+    private static final byte[] value3 = Bytes.toBytes("value3");
+  }
+  
+  private static final String TEST_ATTRIBUTE = "TEST_ATTRIBUTE";
+
+  public static class TestRegionObserver extends BaseRegionObserver {
+    @Override
+    public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
+                       final Put put, final WALEdit edit,
+                       final Durability durability) throws IOException {
+      if (put.getAttribute(TEST_ATTRIBUTE) == null) {
+        throw new DoNotRetryIOException("Put should preserve attributes");
+      }
+      if (put.getDurability() != Durability.USE_DEFAULT) {
+        throw new DoNotRetryIOException("Durability is not propagated correctly");
+      }
+    }
+
+    @Override
+    public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
+                          final Delete delete, final WALEdit edit,
+                          final Durability durability) throws IOException {
+      if (delete.getAttribute(TEST_ATTRIBUTE) == null) {
+        throw new DoNotRetryIOException("Delete should preserve attributes");
+      }
+      if (delete.getDurability() != Durability.USE_DEFAULT) {
+        throw new DoNotRetryIOException("Durability is not propagated correctly");
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    testUtil = new HBaseTestingUtility();
+    conf = testUtil.getConfiguration();
+
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tmpFolder.newFolder().getAbsolutePath());
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+
+    conf.unset(TxConstants.Manager.CFG_TX_HDFS_USER);
+    conf.set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, tmpFolder.newFolder().getAbsolutePath());
+
+    conf.setLong(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5);
+    
+    // Tune down the connection thread pool size
+    conf.setInt("hbase.hconnection.threads.core", 5);
+    conf.setInt("hbase.hconnection.threads.max", 10);
+    // Tunn down handler threads in regionserver
+    conf.setInt("hbase.regionserver.handler.count", 10);
+
+    // Set to random port
+    conf.setInt("hbase.master.port", 0);
+    conf.setInt("hbase.master.info.port", 0);
+    conf.setInt("hbase.regionserver.port", 0);
+    conf.setInt("hbase.regionserver.info.port", 0);
+
+    testUtil.startMiniCluster();
+    hBaseAdmin = testUtil.getHBaseAdmin();
+    txStateStorage = new HDFSTransactionStateStorage(conf, new SnapshotCodecProvider(conf), new TxMetricsCollector());
+    txManager = new TransactionManager(conf, txStateStorage, new TxMetricsCollector());
+    txManager.startAndWait();
+  }
+
+  @AfterClass
+  public static void shutdownAfterClass() throws Exception {
+    if (txManager != null) {
+      txManager.stopAndWait();
+    }
+  }
+
+  @Before
+  public void setupBeforeTest() throws Exception {
+    hTable = createTable(TestBytes.table, new byte[][]{TestBytes.family});
+    transactionAwareHTable = new TransactionAwareHTable(hTable);
+    transactionContext = new TransactionContext(new InMemoryTxSystemClient(txManager), transactionAwareHTable);
+  }
+
+  @After
+  public void shutdownAfterTest() throws IOException {
+    hBaseAdmin.disableTable(TestBytes.table);
+    hBaseAdmin.deleteTable(TestBytes.table);
+  }
+
+  /**
+   * Test transactional put and get requests.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testValidTransactionalPutAndGet() throws Exception {
+    transactionContext.start();
+    Put put = new Put(TestBytes.row);
+    put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+    transactionAwareHTable.put(put);
+    transactionContext.finish();
+
+    transactionContext.start();
+    Result result = transactionAwareHTable.get(new Get(TestBytes.row));
+    transactionContext.finish();
+
+    byte[] value = result.getValue(TestBytes.family, TestBytes.qualifier);
+    assertArrayEquals(TestBytes.value, value);
+  }
+
+  /**
+   * Test aborted put requests, that must be rolled back.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbortedTransactionPutAndGet() throws Exception {
+    transactionContext.start();
+    Put put = new Put(TestBytes.row);
+    put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+    transactionAwareHTable.put(put);
+
+    transactionContext.abort();
+
+    transactionContext.start();
+    Result result = transactionAwareHTable.get(new Get(TestBytes.row));
+    transactionContext.finish();
+    byte[] value = result.getValue(TestBytes.family, TestBytes.qualifier);
+    assertArrayEquals(value, null);
+  }
+
+  /**
+   * Test transactional delete operations.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testValidTransactionalDelete() throws Exception {
+    try (HTable hTable = createTable(Bytes.toBytes("TestValidTransactionalDelete"),
+                                     new byte[][]{TestBytes.family, TestBytes.family2})) {
+      TransactionAwareHTable txTable = new TransactionAwareHTable(hTable);
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+
+      txContext.start();
+      Put put = new Put(TestBytes.row);
+      put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+      put.add(TestBytes.family2, TestBytes.qualifier, TestBytes.value2);
+      txTable.put(put);
+      txContext.finish();
+
+      txContext.start();
+      Result result = txTable.get(new Get(TestBytes.row));
+      txContext.finish();
+      byte[] value = result.getValue(TestBytes.family, TestBytes.qualifier);
+      assertArrayEquals(TestBytes.value, value);
+      value = result.getValue(TestBytes.family2, TestBytes.qualifier);
+      assertArrayEquals(TestBytes.value2, value);
+
+      // test full row delete
+      txContext.start();
+      Delete delete = new Delete(TestBytes.row);
+      txTable.delete(delete);
+      txContext.finish();
+
+      txContext.start();
+      result = txTable.get(new Get(TestBytes.row));
+      txContext.finish();
+      assertTrue(result.isEmpty());
+      
+      // test column delete
+      // load 10 rows
+      txContext.start();
+      int rowCount = 10;
+      for (int i = 0; i < rowCount; i++) {
+        Put p = new Put(Bytes.toBytes("row" + i));
+        for (int j = 0; j < 10; j++) {
+          p.add(TestBytes.family, Bytes.toBytes(j), TestBytes.value);
+        }
+        txTable.put(p);
+      }
+      txContext.finish();
+
+      // verify loaded rows
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Get g = new Get(Bytes.toBytes("row" + i));
+        Result r = txTable.get(g);
+        assertFalse(r.isEmpty());
+        for (int j = 0; j < 10; j++) {
+          assertArrayEquals(TestBytes.value, r.getValue(TestBytes.family, Bytes.toBytes(j)));
+        }
+      }
+      txContext.finish();
+
+      // delete odds columns from odd rows and even columns from even rows
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Delete d = new Delete(Bytes.toBytes("row" + i));
+        for (int j = 0; j < 10; j++) {
+          if (i % 2 == j % 2) {
+            d.deleteColumns(TestBytes.family, Bytes.toBytes(j));
+          }
+        }
+        txTable.delete(d);
+      }
+      txContext.finish();
+
+      // verify deleted columns
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Get g = new Get(Bytes.toBytes("row" + i));
+        Result r = txTable.get(g);
+        assertEquals(5, r.size());
+        for (Map.Entry<byte[], byte[]> entry : r.getFamilyMap(TestBytes.family).entrySet()) {
+          int col = Bytes.toInt(entry.getKey());
+          // each row should only have the opposite mod (odd=even, even=odd)
+          assertNotEquals(i % 2, col % 2);
+          assertArrayEquals(TestBytes.value, entry.getValue());
+        }
+      }
+      txContext.finish();
+
+      // test family delete
+      // load 10 rows
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Put p = new Put(Bytes.toBytes("famrow" + i));
+        p.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+        p.add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value2);
+        txTable.put(p);
+      }
+      txContext.finish();
+
+      // verify all loaded rows
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Get g = new Get(Bytes.toBytes("famrow" + i));
+        Result r = txTable.get(g);
+        assertEquals(2, r.size());
+        assertArrayEquals(TestBytes.value, r.getValue(TestBytes.family, TestBytes.qualifier));
+        assertArrayEquals(TestBytes.value2, r.getValue(TestBytes.family2, TestBytes.qualifier2));
+      }
+      txContext.finish();
+
+      // delete family1 for even rows, family2 for odd rows
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Delete d = new Delete(Bytes.toBytes("famrow" + i));
+        d.deleteFamily((i % 2 == 0) ? TestBytes.family : TestBytes.family2);
+        txTable.delete(d);
+      }
+      txContext.finish();
+
+      // verify deleted families
+      txContext.start();
+      for (int i = 0; i < rowCount; i++) {
+        Get g = new Get(Bytes.toBytes("famrow" + i));
+        Result r = txTable.get(g);
+        assertEquals(1, r.size());
+        if (i % 2 == 0) {
+          assertNull(r.getValue(TestBytes.family, TestBytes.qualifier));
+          assertArrayEquals(TestBytes.value2, r.getValue(TestBytes.family2, TestBytes.qualifier2));
+        } else {
+          assertArrayEquals(TestBytes.value, r.getValue(TestBytes.family, TestBytes.qualifier));
+          assertNull(r.getValue(TestBytes.family2, TestBytes.qualifier2));
+        }
+      }
+      txContext.finish();
+    }
+  }
+
+  /**
+   * Test that put and delete attributes are preserved
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAttributesPreserved() throws Exception {
+    HTable hTable = createTable(Bytes.toBytes("TestAttributesPreserved"),
+        new byte[][]{TestBytes.family, TestBytes.family2}, false,
+        Lists.newArrayList(TransactionProcessor.class.getName(), TestRegionObserver.class.getName()));
+    try {
+      TransactionAwareHTable txTable = new TransactionAwareHTable(hTable);
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+
+      txContext.start();
+      Put put = new Put(TestBytes.row);
+      put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+      put.add(TestBytes.family2, TestBytes.qualifier, TestBytes.value2);
+      // set an attribute on the put, TestRegionObserver will verify it still exists
+      put.setAttribute(TEST_ATTRIBUTE, new byte[]{});
+      txTable.put(put);
+      txContext.finish();
+
+      txContext.start();
+      Result result = txTable.get(new Get(TestBytes.row));
+      txContext.finish();
+      byte[] value = result.getValue(TestBytes.family, TestBytes.qualifier);
+      assertArrayEquals(TestBytes.value, value);
+      value = result.getValue(TestBytes.family2, TestBytes.qualifier);
+      assertArrayEquals(TestBytes.value2, value);
+
+      // test full row delete, TestRegionObserver will verify it still exists
+      txContext.start();
+      Delete delete = new Delete(TestBytes.row);
+      delete.setAttribute(TEST_ATTRIBUTE, new byte[]{});
+      txTable.delete(delete);
+      txContext.finish();
+
+      txContext.start();
+      result = txTable.get(new Get(TestBytes.row));
+      txContext.finish();
+      assertTrue(result.isEmpty());
+    } finally {
+        hTable.close();
+      }
+    }
+  
+  @Test
+  public void testFamilyDeleteWithCompaction() throws Exception {
+    HTable hTable = createTable(Bytes.toBytes("TestFamilyDeleteWithCompaction"),
+                                new byte[][]{TestBytes.family, TestBytes.family2});
+    try {
+      TransactionAwareHTable txTable = new TransactionAwareHTable(hTable, ConflictDetection.ROW);
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+            
+      txContext.start();
+      Put put = new Put(TestBytes.row);
+      put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+      txTable.put(put);
+      
+      put = new Put(TestBytes.row2);
+      put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+      txTable.put(put);
+      txContext.finish();
+      
+      txContext.start();
+      Result result = txTable.get(new Get(TestBytes.row));
+      txContext.finish();
+      assertFalse(result.isEmpty());
+      
+      txContext.start();
+      // test family delete with ConflictDetection.ROW (as ConflictDetection.COLUMN converts this to a column delete)
+      Delete delete = new Delete(TestBytes.row);
+      delete.deleteFamily(TestBytes.family);
+      txTable.delete(delete);
+      txContext.finish();
+      
+      txContext.start();
+      result = txTable.get(new Get(TestBytes.row));
+      txContext.finish();
+      assertTrue(result.isEmpty());
+      
+      boolean compactionDone = false;
+      int count = 0;
+      while (count++ < 12 && !compactionDone) {
+         // run major compaction and verify the row was removed
+         HBaseAdmin hbaseAdmin = testUtil.getHBaseAdmin();
+         hbaseAdmin.flush("TestFamilyDeleteWithCompaction");
+         hbaseAdmin.majorCompact("TestFamilyDeleteWithCompaction");
+         hbaseAdmin.close();
+         Thread.sleep(5000L);
+         
+         Scan scan = new Scan();
+         scan.setStartRow(TestBytes.row);
+         scan.setStopRow(Bytes.add(TestBytes.row, new byte[] { 0 }));
+         scan.setRaw(true);
+         
+         ResultScanner scanner = hTable.getScanner(scan);
+         compactionDone = scanner.next() == null;
+         scanner.close();
+      }
+      assertTrue("Compaction should have removed the row", compactionDone);
+      
+      Scan scan = new Scan();
+      scan.setStartRow(TestBytes.row2);
+      scan.setStopRow(Bytes.add(TestBytes.row2, new byte[] { 0 }));
+      scan.setRaw(true);
+      
+      ResultScanner scanner = hTable.getScanner(scan);
+      Result res = scanner.next();
+      assertNotNull(res);
+      } finally {
+        hTable.close();
+      }
+  }
+  
+  /**
+   * Test aborted transactional delete requests, that must be rolled back.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAbortedTransactionalDelete() throws Exception {
+    transactionContext.start();
+    Put put = new Put(TestBytes.row);
+    put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+    transactionAwareHTable.put(put);
+    transactionContext.finish();
+
+    transactionContext.start();
+    Result result = transactionAwareHTable.get(new Get(TestBytes.row));
+    transactionContext.finish();
+    byte[] value = result.getValue(TestBytes.family, TestBytes.qualifier);
+    assertArrayEquals(TestBytes.value, value);
+
+    transactionContext.start();
+    Delete delete = new Delete(TestBytes.row);
+    transactionAwareHTable.delete(delete);
+    transactionContext.abort();
+
+    transactionContext.start();
+    result = transactionAwareHTable.get(new Get(TestBytes.row));
+    transactionContext.finish();
+    value = result.getValue(TestBytes.family, TestBytes.qualifier);
+    assertArrayEquals(TestBytes.value, value);
+  }
+
+  private void testDeleteRollback(TxConstants.ConflictDetection conflictDetection) throws Exception {
+    String tableName = String.format("%s%s", "TestColFamilyDelete", conflictDetection);
+    HTable hTable = createTable(Bytes.toBytes(tableName), new byte[][]{TestBytes.family});
+    try (TransactionAwareHTable txTable = new TransactionAwareHTable(hTable, conflictDetection)) {
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+      txContext.start();
+      txTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+      txContext.finish();
+
+      // Start a tx, delete the row and then abort the tx
+      txContext.start();
+      txTable.delete(new Delete(TestBytes.row));
+      txContext.abort();
+
+      // Start a tx, delete a column family and then abort the tx
+      txContext.start();
+      txTable.delete(new Delete(TestBytes.row).deleteFamily(TestBytes.family));
+      txContext.abort();
+
+      // Above operations should have no effect on the row, since they were aborted
+      txContext.start();
+      Get get = new Get(TestBytes.row);
+      Result result = txTable.get(get);
+      assertFalse(result.isEmpty());
+      assertArrayEquals(TestBytes.value, result.getValue(TestBytes.family, TestBytes.qualifier));
+      txContext.finish();
+    }
+  }
+
+  @Test
+  public void testDeleteRollback() throws Exception {
+    testDeleteRollback(TxConstants.ConflictDetection.ROW);
+    testDeleteRollback(TxConstants.ConflictDetection.COLUMN);
+    testDeleteRollback(TxConstants.ConflictDetection.NONE);
+  }
+
+  @Test
+  public void testMultiColumnFamilyRowDeleteRollback() throws Exception {
+    HTable hTable = createTable(Bytes.toBytes("TestMultColFam"), new byte[][] {TestBytes.family, TestBytes.family2});
+    try (TransactionAwareHTable txTable = new TransactionAwareHTable(hTable, TxConstants.ConflictDetection.ROW)) {
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+      txContext.start();
+      txTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+      txContext.finish();
+
+      txContext.start();
+      //noinspection ConstantConditions
+      txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_ALL);
+      Result result = txTable.get(new Get(TestBytes.row));
+      Assert.assertEquals(1, result.getFamilyMap(TestBytes.family).size());
+      Assert.assertEquals(0, result.getFamilyMap(TestBytes.family2).size());
+      txContext.finish();
+
+      //Start a tx, delete the row and then abort the tx
+      txContext.start();
+      txTable.delete(new Delete(TestBytes.row));
+      txContext.abort();
+
+      //Start a tx and scan all the col families to make sure none of them have delete markers
+      txContext.start();
+      txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_ALL);
+      result = txTable.get(new Get(TestBytes.row));
+      Assert.assertEquals(1, result.getFamilyMap(TestBytes.family).size());
+      Assert.assertEquals(0, result.getFamilyMap(TestBytes.family2).size());
+      txContext.finish();
+    }
+  }
+
+  @Test
+  public void testRowDelete() throws Exception {
+    HTable hTable = createTable(Bytes.toBytes("TestRowDelete"), new byte[][]{TestBytes.family, TestBytes.family2});
+    try (TransactionAwareHTable txTable = new TransactionAwareHTable(hTable, TxConstants.ConflictDetection.ROW)) {
+      TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+
+      // Test 1: full row delete
+      txContext.start();
+      txTable.put(new Put(TestBytes.row)
+                    .add(TestBytes.family, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family, TestBytes.qualifier2, TestBytes.value2)
+                    .add(TestBytes.family2, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value2));
+      txContext.finish();
+
+      txContext.start();
+      Get get = new Get(TestBytes.row);
+      Result result = txTable.get(get);
+      assertFalse(result.isEmpty());
+      assertArrayEquals(TestBytes.value, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(TestBytes.value2, result.getValue(TestBytes.family, TestBytes.qualifier2));
+      assertArrayEquals(TestBytes.value, result.getValue(TestBytes.family2, TestBytes.qualifier));
+      assertArrayEquals(TestBytes.value2, result.getValue(TestBytes.family2, TestBytes.qualifier2));
+      txContext.finish();
+
+      // delete entire row
+      txContext.start();
+      txTable.delete(new Delete(TestBytes.row));
+      txContext.finish();
+
+      // verify row is now empty
+      txContext.start();
+      result = txTable.get(new Get(TestBytes.row));
+      assertTrue(result.isEmpty());
+
+      // verify row is empty for explicit column retrieval
+      result = txTable.get(new Get(TestBytes.row)
+                             .addColumn(TestBytes.family, TestBytes.qualifier)
+                             .addFamily(TestBytes.family2));
+      assertTrue(result.isEmpty());
+
+      // verify row is empty for scan
+      ResultScanner scanner = txTable.getScanner(new Scan(TestBytes.row));
+      assertNull(scanner.next());
+      scanner.close();
+
+      // verify row is empty for scan with explicit column
+      scanner = txTable.getScanner(new Scan(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier2));
+      assertNull(scanner.next());
+      scanner.close();
+      txContext.finish();
+
+      // write swapped values to one column per family
+      txContext.start();
+      txTable.put(new Put(TestBytes.row)
+                    .add(TestBytes.family, TestBytes.qualifier, TestBytes.value2)
+                    .add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value));
+      txContext.finish();
+
+      // verify new values appear
+      txContext.start();
+      result = txTable.get(new Get(TestBytes.row));
+      assertFalse(result.isEmpty());
+      assertEquals(2, result.size());
+      assertArrayEquals(TestBytes.value2, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(TestBytes.value, result.getValue(TestBytes.family2, TestBytes.qualifier2));
+
+      scanner = txTable.getScanner(new Scan(TestBytes.row));
+      Result result1 = scanner.next();
+      assertNotNull(result1);
+      assertFalse(result1.isEmpty());
+      assertEquals(2, result1.size());
+      assertArrayEquals(TestBytes.value2, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(TestBytes.value, result.getValue(TestBytes.family2, TestBytes.qualifier2));
+      scanner.close();
+      txContext.finish();
+
+      // Test 2: delete of first column family
+      txContext.start();
+      txTable.put(new Put(TestBytes.row2)
+                    .add(TestBytes.family, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family, TestBytes.qualifier2, TestBytes.value2)
+                    .add(TestBytes.family2, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value2));
+      txContext.finish();
+
+      txContext.start();
+      txTable.delete(new Delete(TestBytes.row2).deleteFamily(TestBytes.family));
+      txContext.finish();
+
+      txContext.start();
+      Result fam1Result = txTable.get(new Get(TestBytes.row2));
+      assertFalse(fam1Result.isEmpty());
+      assertEquals(2, fam1Result.size());
+      assertArrayEquals(TestBytes.value, fam1Result.getValue(TestBytes.family2, TestBytes.qualifier));
+      assertArrayEquals(TestBytes.value2, fam1Result.getValue(TestBytes.family2, TestBytes.qualifier2));
+      txContext.finish();
+
+      // Test 3: delete of second column family
+      txContext.start();
+      txTable.put(new Put(TestBytes.row3)
+                    .add(TestBytes.family, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family, TestBytes.qualifier2, TestBytes.value2)
+                    .add(TestBytes.family2, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value2));
+      txContext.finish();
+
+      txContext.start();
+      txTable.delete(new Delete(TestBytes.row3).deleteFamily(TestBytes.family2));
+      txContext.finish();
+
+      txContext.start();
+      Result fam2Result = txTable.get(new Get(TestBytes.row3));
+      assertFalse(fam2Result.isEmpty());
+      assertEquals(2, fam2Result.size());
+      assertArrayEquals(TestBytes.value, fam2Result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(TestBytes.value2, fam2Result.getValue(TestBytes.family, TestBytes.qualifier2));
+      txContext.finish();
+
+      // Test 4: delete specific rows in a range
+      txContext.start();
+      for (int i = 0; i < 10; i++) {
+        txTable.put(new Put(Bytes.toBytes("z" + i))
+                      .add(TestBytes.family, TestBytes.qualifier, Bytes.toBytes(i))
+                      .add(TestBytes.family2, TestBytes.qualifier2, Bytes.toBytes(i)));
+      }
+      txContext.finish();
+
+      txContext.start();
+      // delete odd rows
+      for (int i = 1; i < 10; i += 2) {
+        txTable.delete(new Delete(Bytes.toBytes("z" + i)));
+      }
+      txContext.finish();
+
+      txContext.start();
+      int cnt = 0;
+      ResultScanner zScanner = txTable.getScanner(new Scan(Bytes.toBytes("z0")));
+      Result res;
+      while ((res = zScanner.next()) != null) {
+        assertFalse(res.isEmpty());
+        assertArrayEquals(Bytes.toBytes("z" + cnt), res.getRow());
+        assertArrayEquals(Bytes.toBytes(cnt), res.getValue(TestBytes.family, TestBytes.qualifier));
+        assertArrayEquals(Bytes.toBytes(cnt), res.getValue(TestBytes.family2, TestBytes.qualifier2));
+        cnt += 2;
+      }
+
+      // Test 5: delete prior writes in the same transaction
+      txContext.start();
+      txTable.put(new Put(TestBytes.row4)
+                    .add(TestBytes.family, TestBytes.qualifier, TestBytes.value)
+                    .add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value2));
+      txTable.delete(new Delete(TestBytes.row4));
+      txContext.finish();
+
+      txContext.start();
+      Result row4Result = txTable.get(new Get(TestBytes.row4));
+      assertTrue(row4Result.isEmpty());
+      txContext.finish();
+    }
+  }
+
+  /**
+   * Expect an exception since a transaction hasn't been started.
+   *
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testTransactionlessFailure() throws Exception {
+    transactionAwareHTable.get(new Get(TestBytes.row));
+  }
+
+  /**
+   * Tests that each transaction can see its own persisted writes, while not seeing writes from other
+   * in-progress transactions.
+   */
+  @Test
+  public void testReadYourWrites() throws Exception {
+    // In-progress tx1: started before our main transaction
+    HTable hTable1 = new HTable(testUtil.getConfiguration(), TestBytes.table);
+    TransactionAwareHTable txHTable1 = new TransactionAwareHTable(hTable1);
+    TransactionContext inprogressTxContext1 = new TransactionContext(new InMemoryTxSystemClient(txManager), txHTable1);
+
+    // In-progress tx2: started while our main transaction is running
+    HTable hTable2 = new HTable(testUtil.getConfiguration(), TestBytes.table);
+    TransactionAwareHTable txHTable2 = new TransactionAwareHTable(hTable2);
+    TransactionContext inprogressTxContext2 = new TransactionContext(new InMemoryTxSystemClient(txManager), txHTable2);
+
+    // create an in-progress write that should be ignored
+    byte[] col2 = Bytes.toBytes("col2");
+    inprogressTxContext1.start();
+    Put putCol2 = new Put(TestBytes.row);
+    byte[] valueCol2 = Bytes.toBytes("writing in progress");
+    putCol2.add(TestBytes.family, col2, valueCol2);
+    txHTable1.put(putCol2);
+
+    // start a tx and write a value to test reading in same tx
+    transactionContext.start();
+    Put put = new Put(TestBytes.row);
+    byte[] value = Bytes.toBytes("writing");
+    put.add(TestBytes.family, TestBytes.qualifier, value);
+    transactionAwareHTable.put(put);
+
+    // test that a write from a tx started after the first is not visible
+    inprogressTxContext2.start();
+    Put put2 = new Put(TestBytes.row);
+    byte[] value2 = Bytes.toBytes("writing2");
+    put2.add(TestBytes.family, TestBytes.qualifier, value2);
+    txHTable2.put(put2);
+
+    Get get = new Get(TestBytes.row);
+    Result row = transactionAwareHTable.get(get);
+    assertFalse(row.isEmpty());
+    byte[] col1Value = row.getValue(TestBytes.family, TestBytes.qualifier);
+    Assert.assertNotNull(col1Value);
+    Assert.assertArrayEquals(value, col1Value);
+    // write from in-progress transaction should not be visible
+    byte[] col2Value = row.getValue(TestBytes.family, col2);
+    assertNull(col2Value);
+
+    // commit in-progress transaction, should still not be visible
+    inprogressTxContext1.finish();
+
+    get = new Get(TestBytes.row);
+    row = transactionAwareHTable.get(get);
+    assertFalse(row.isEmpty());
+    col2Value = row.getValue(TestBytes.family, col2);
+    assertNull(col2Value);
+
+    transactionContext.finish();
+
+    inprogressTxContext2.abort();
+  }
+
+  @Test
+  public void testRowLevelConflictDetection() throws Exception {
+    TransactionAwareHTable txTable1 = new TransactionAwareHTable(new HTable(conf, TestBytes.table),
+        TxConstants.ConflictDetection.ROW);
+    TransactionContext txContext1 = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable1);
+
+    TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table),
+        TxConstants.ConflictDetection.ROW);
+    TransactionContext txContext2 = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable2);
+
+    byte[] row1 = Bytes.toBytes("row1");
+    byte[] row2 = Bytes.toBytes("row2");
+    byte[] col1 = Bytes.toBytes("c1");
+    byte[] col2 = Bytes.toBytes("c2");
+    byte[] val1 = Bytes.toBytes("val1");
+    byte[] val2 = Bytes.toBytes("val2");
+
+    // test that concurrent writing to different rows succeeds
+    txContext1.start();
+    txTable1.put(new Put(row1).add(TestBytes.family, col1, val1));
+
+    txContext2.start();
+    txTable2.put(new Put(row2).add(TestBytes.family, col1, val2));
+
+    // should be no conflicts
+    txContext1.finish();
+    txContext2.finish();
+
+    transactionContext.start();
+    Result res = transactionAwareHTable.get(new Get(row1));
+    assertFalse(res.isEmpty());
+    Cell cell = res.getColumnLatestCell(TestBytes.family, col1);
+    assertNotNull(cell);
+    assertArrayEquals(val1, CellUtil.cloneValue(cell));
+
+    res = transactionAwareHTable.get(new Get(row2));
+    assertFalse(res.isEmpty());
+    cell = res.getColumnLatestCell(TestBytes.family, col1);
+    assertNotNull(cell);
+    assertArrayEquals(val2, CellUtil.cloneValue(cell));
+    transactionContext.finish();
+
+    // test that writing to different columns in the same row fails
+    txContext1.start();
+    txTable1.put(new Put(row1).add(TestBytes.family, col1, val2));
+
+    txContext2.start();
+    txTable2.put(new Put(row1).add(TestBytes.family, col2, val2));
+
+    txContext1.finish();
+    try {
+      txContext2.finish();
+      fail("txContext2 should have encountered a row-level conflict during commit");
+    } catch (TransactionConflictException tce) {
+      txContext2.abort();
+    }
+
+    transactionContext.start();
+    res = transactionAwareHTable.get(new Get(row1));
+    assertFalse(res.isEmpty());
+    cell = res.getColumnLatestCell(TestBytes.family, col1);
+    assertNotNull(cell);
+    // should now be val2
+    assertArrayEquals(val2, CellUtil.cloneValue(cell));
+
+    cell = res.getColumnLatestCell(TestBytes.family, col2);
+    // col2 should not be visible due to conflict
+    assertNull(cell);
+    transactionContext.finish();
+
+    // test that writing to the same column in the same row fails
+    txContext1.start();
+    txTable1.put(new Put(row2).add(TestBytes.family, col2, val1));
+
+    txContext2.start();
+    txTable2.put(new Put(row2).add(TestBytes.family, col2, val2));
+
+    txContext1.finish();
+    try {
+      txContext2.finish();
+      fail("txContext2 should have encountered a row and column level conflict during commit");
+    } catch (TransactionConflictException tce) {
+      txContext2.abort();
+    }
+
+    transactionContext.start();
+    res = transactionAwareHTable.get(new Get(row2));
+    assertFalse(res.isEmpty());
+    cell = res.getColumnLatestCell(TestBytes.family, col2);
+    assertNotNull(cell);
+    // should now be val1
+    assertArrayEquals(val1, CellUtil.cloneValue(cell));
+    transactionContext.finish();
+
+    // verify change set that is being reported only on rows
+    txContext1.start();
+    txTable1.put(new Put(row1).add(TestBytes.family, col1, val1));
+    txTable1.put(new Put(row2).add(TestBytes.family, col2, val2));
+
+    Collection<byte[]> changeSet = txTable1.getTxChanges();
+    assertNotNull(changeSet);
+    assertEquals(2, changeSet.size());
+    assertTrue(changeSet.contains(txTable1.getChangeKey(row1, null, null)));
+    assertTrue(changeSet.contains(txTable1.getChangeKey(row2, null, null)));
+    txContext1.finish();
+  }
+
+  @Test
+  public void testNoneLevelConflictDetection() throws Exception {
+    InMemoryTxSystemClient txClient = new InMemoryTxSystemClient(txManager);
+    TransactionAwareHTable txTable1 = new TransactionAwareHTable(new HTable(conf, TestBytes.table),
+        TxConstants.ConflictDetection.NONE);
+    TransactionContext txContext1 = new TransactionContext(txClient, txTable1);
+
+    TransactionAwareHTable txTable2 = new TransactionAwareHTable(new HTable(conf, TestBytes.table),
+        TxConstants.ConflictDetection.NONE);
+    TransactionContext txContext2 = new TransactionContext(txClient, txTable2);
+
+    // overlapping writes to the same row + column should not conflict
+
+    txContext1.start();
+    txTable1.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+
+    // changes should not be visible yet
+    txContext2.start();
+    Result row = txTable2.get(new Get(TestBytes.row));
+    assertTrue(row.isEmpty());
+
+    txTable2.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value2));
+
+    // both commits should succeed
+    txContext1.finish();
+    txContext2.finish();
+
+    txContext1.start();
+    row = txTable1.get(new Get(TestBytes.row));
+    assertFalse(row.isEmpty());
+    assertArrayEquals(TestBytes.value2, row.getValue(TestBytes.family, TestBytes.qualifier));
+    txContext1.finish();
+
+    // transaction abort should still rollback changes
+
+    txContext1.start();
+    txTable1.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+    txContext1.abort();
+
+    // changes to row2 should be rolled back
+    txContext2.start();
+    Result row2 = txTable2.get(new Get(TestBytes.row2));
+    assertTrue(row2.isEmpty());
+    txContext2.finish();
+
+    // transaction invalidate should still make changes invisible
+
+    txContext1.start();
+    Transaction tx1 = txContext1.getCurrentTransaction();
+    txTable1.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+    assertNotNull(tx1);
+    txClient.invalidate(tx1.getWritePointer());
+
+    // changes to row2 should be rolled back
+    txContext2.start();
+    Result row3 = txTable2.get(new Get(TestBytes.row3));
+    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.getTransactionId(), postCheckpointTx.getTransactionId());
+    assertNotEquals(origTx.getWritePointer(), postCheckpointTx.getWritePointer());
+    long[] checkpointPtrs = postCheckpointTx.getCheckpointWritePointers();
+    assertEquals(1, checkpointPtrs.length);
+    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.getTransactionId(), postCheckpointTx2.getTransactionId());
+    assertNotEquals(postCheckpointTx.getWritePointer(), postCheckpointTx2.getWritePointer());
+    long[] checkpointPtrs2 = postCheckpointTx2.getCheckpointWritePointers();
+    assertEquals(2, checkpointPtrs2.length);
+    assertEquals(postCheckpointTx.getWritePointer(), checkpointPtrs2[0]);
+    assertEquals(postCheckpointTx2.getWritePointer(), 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
+    transactionContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT);
+    Get get = new Get(TestBytes.row);
+    verifyRow(transactionAwareHTable, get, TestBytes.value);
+    get = new Get(TestBytes.row2);
+    verifyRow(transactionAwareHTable, get, TestBytes.value2);
+    get = new Get(TestBytes.row3);
+    verifyRow(transactionAwareHTable, get, null);
+
+    // test scan results excluding current write pointer
+    Scan scan = new Scan();
+    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();
+    transactionContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT);
+
+    // commit transaction, verify writes are visible
+    transactionContext.finish();
+
+    transactionContext.start();
+    verifyRow(transactionAwareHTable, TestBytes.row, TestBytes.value);
+    verifyRow(transactionAwareHTable, TestBytes.row2, TestBytes.value2);
+    verifyRow(transactionAwareHTable, TestBytes.row3, TestBytes.value);
+    transactionContext.finish();
+  }
+
+  @Test
+  public void testInProgressCheckpoint() 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));
+
+    // 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);
+    txContext2.finish();
+    txTable2.close();
+
+    transactionContext.finish();
+
+    // verify writes are visible after commit
+    transactionContext.start();
+    verifyRow(transactionAwareHTable, TestBytes.row, TestBytes.value);
+    verifyRow(transactionAwareHTable, TestBytes.row2, TestBytes.value2);
+    transactionContext.finish();
+  }
+
+  @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().getTransactionId());
+
+    // 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.getWritePointer()));
+    assertTrue(newTx.isExcluded(checkpointTx1.getWritePointer()));
+    assertTrue(newTx.isExcluded(checkpointTx2.getWritePointer()));
+
+    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();
+  }
+
+  @Test
+  public void testExistingData() throws Exception {
+    byte[] val11 = Bytes.toBytes("val11");
+    byte[] val12 = Bytes.toBytes("val12");
+    byte[] val21 = Bytes.toBytes("val21");
+    byte[] val22 = Bytes.toBytes("val22");
+    byte[] val31 = Bytes.toBytes("val31");
+    byte[] val111 = Bytes.toBytes("val111");
+
+    TransactionAwareHTable txTable =
+      new TransactionAwareHTable(createTable(Bytes.toBytes("testExistingData"), new byte[][]{TestBytes.family}, true, 
+      Collections.singletonList(TransactionProcessor.class.getName())));
+    TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+
+    // Add some pre-existing, non-transactional data
+    HTable nonTxTable = new HTable(testUtil.getConfiguration(), txTable.getTableName());
+    nonTxTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, val11));
+    nonTxTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier2, val12));
+    nonTxTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, val21));
+    nonTxTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier2, val22));
+    nonTxTable.put(new Put(TestBytes.row4).add(TestBytes.family, TxConstants.FAMILY_DELETE_QUALIFIER,
+                                               HConstants.EMPTY_BYTE_ARRAY));
+    nonTxTable.put(new Put(TestBytes.row4).add(TestBytes.family, TestBytes.qualifier, HConstants.EMPTY_BYTE_ARRAY));
+    nonTxTable.flushCommits();
+
+    // Add transactional data
+    txContext.start();
+    txTable.put(new Put(TestBytes.row3).add(TestBytes.family, TestBytes.qualifier, val31));
+    txContext.finish();
+
+    txContext.start();
+    // test get
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier), val11);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2), val12);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier), val21);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier2), val22);
+    verifyRow(txTable, new Get(TestBytes.row3).addColumn(TestBytes.family, TestBytes.qualifier), val31);
+    verifyRow(txTable, new Get(TestBytes.row4).addColumn(TestBytes.family, TxConstants.FAMILY_DELETE_QUALIFIER),
+              HConstants.EMPTY_BYTE_ARRAY);
+    verifyRow(txTable, new Get(TestBytes.row4).addColumn(TestBytes.family, TestBytes.qualifier),
+              HConstants.EMPTY_BYTE_ARRAY);
+
+    // test scan
+    try (ResultScanner scanner = txTable.getScanner(new Scan())) {
+      Result result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row, result.getRow());
+      assertArrayEquals(val11, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(val12, result.getValue(TestBytes.family, TestBytes.qualifier2));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row2, result.getRow());
+      assertArrayEquals(val21, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(val22, result.getValue(TestBytes.family, TestBytes.qualifier2));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row3, result.getRow());
+      assertArrayEquals(val31, result.getValue(TestBytes.family, TestBytes.qualifier));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row4, result.getRow());
+      assertArrayEquals(HConstants.EMPTY_BYTE_ARRAY, result.getValue(TestBytes.family,
+                                                                     TxConstants.FAMILY_DELETE_QUALIFIER));
+      assertArrayEquals(HConstants.EMPTY_BYTE_ARRAY, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertNull(scanner.next());
+    }
+    txContext.finish();
+
+    // test update and delete
+    txContext.start();
+    txTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, val111));
+    txTable.delete(new Delete(TestBytes.row2).deleteColumns(TestBytes.family, TestBytes.qualifier));
+    txContext.finish();
+
+    txContext.start();
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier), val111);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2), val12);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier), null);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier2), val22);
+    verifyRow(txTable, new Get(TestBytes.row3).addColumn(TestBytes.family, TestBytes.qualifier), val31);
+    verifyRow(txTable, new Get(TestBytes.row4).addColumn(TestBytes.family, TxConstants.FAMILY_DELETE_QUALIFIER),
+              HConstants.EMPTY_BYTE_ARRAY);
+    verifyRow(txTable, new Get(TestBytes.row4).addColumn(TestBytes.family, TestBytes.qualifier),
+              HConstants.EMPTY_BYTE_ARRAY);
+    txContext.finish();
+
+    // test scan
+    txContext.start();
+    try (ResultScanner scanner = txTable.getScanner(new Scan())) {
+      Result result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row, result.getRow());
+      assertArrayEquals(val111, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(val12, result.getValue(TestBytes.family, TestBytes.qualifier2));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row2, result.getRow());
+      assertArrayEquals(null, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertArrayEquals(val22, result.getValue(TestBytes.family, TestBytes.qualifier2));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row3, result.getRow());
+      assertArrayEquals(val31, result.getValue(TestBytes.family, TestBytes.qualifier));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row4, result.getRow());
+      assertArrayEquals(HConstants.EMPTY_BYTE_ARRAY, result.getValue(TestBytes.family,
+                                                                     TxConstants.FAMILY_DELETE_QUALIFIER));
+      assertArrayEquals(HConstants.EMPTY_BYTE_ARRAY, result.getValue(TestBytes.family, TestBytes.qualifier));
+      assertNull(scanner.next());
+    }
+    txContext.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 {
+    verifyRows(table, get, expectedValue == null ? null : ImmutableList.of(expectedValue));
+  }
+
+  private void verifyRows(HTableInterface table, Get get, List<byte[]> expectedValues) throws Exception {
+    Result result = table.get(get);
+    if (expectedValues == null) {
+      assertTrue(result.isEmpty());
+    } else {
+      assertFalse(result.isEmpty());
+      byte[] family = TestBytes.family;
+      byte[] col = TestBytes.qualifier;
+      if (get.hasFamilies()) {
+        family = get.getFamilyMap().keySet().iterator().next();
+        col = get.getFamilyMap().get(family).first();
+      }
+      Iterator<Cell> it = result.getColumnCells(family, col).iterator();
+      for (byte[] expectedValue : expectedValues) {
+        Assert.assertTrue(it.hasNext());
+        assertArrayEquals(expectedValue, CellUtil.cloneValue(it.next()));
+      }
+    }
+  }
+
+  private Cell[] getRow(HTableInterface table, Get get) throws Exception {
+    Result result = table.get(get);
+    return result.rawCells();
+  }
+
+  private void verifyScan(HTableInterface table, Scan scan, List<KeyValue> expectedCells) throws Exception {
+    List<Cell> actualCells = new ArrayList<>();
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      Result[] results = scanner.next(expectedCells.size() + 1);
+      for (Result result : results) {
+        actualCells.addAll(Lists.newArrayList(result.rawCells()));
+      }
+      Assert.assertEquals(expectedCells, actualCells);
+    }
+  }
+
+  @Test
+  public void testVisibilityAll() throws Exception {
+    HTable nonTxTable =
+      createTable(Bytes.toBytes("testVisibilityAll"), new byte[][]{TestBytes.family, TestBytes.family2},
+                  true, Collections.singletonList(TransactionProcessor.class.getName()));
+    TransactionAwareHTable txTable =
+      new TransactionAwareHTable(nonTxTable,
+                                 TxConstants.ConflictDetection.ROW); // ROW conflict detection to verify family deletes
+    TransactionContext txContext = new TransactionContext(new InMemoryTxSystemClient(txManager), txTable);
+
+    // start a transaction and create a delete marker
+    txContext.start();
+    //noinspection ConstantConditions
+    long txWp0 = txContext.getCurrentTransaction().getWritePointer();
+    txTable.delete(new Delete(TestBytes.row).deleteColumn(TestBytes.family, TestBytes.qualifier2));
+    txContext.finish();
+
+    // start a new transaction and write some values
+    txContext.start();
+    @SuppressWarnings("ConstantConditions")
+    long txWp1 = txContext.getCurrentTransaction().getWritePointer();
+    txTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+    txTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier2, TestBytes.value2));
+    txTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value));
+    txTable.put(new Put(TestBytes.row).add(TestBytes.family2, TestBytes.qualifier, TestBytes.value));
+    txTable.put(new Put(TestBytes.row).add(TestBytes.family2, TestBytes.qualifier2, TestBytes.value2));
+
+    // verify written data
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier),
+              TestBytes.value);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2),
+              TestBytes.value2);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier),
+              TestBytes.value);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier),
+              TestBytes.value);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier2),
+              TestBytes.value2);
+
+    // checkpoint and make changes to written data now
+    txContext.checkpoint();
+    long txWp2 = txContext.getCurrentTransaction().getWritePointer();
+    // delete a column
+    txTable.delete(new Delete(TestBytes.row).deleteColumn(TestBytes.family, TestBytes.qualifier));
+    // no change to a column
+    txTable.put(new Put(TestBytes.row).add(TestBytes.family, TestBytes.qualifier2, TestBytes.value2));
+    // update a column
+    txTable.put(new Put(TestBytes.row2).add(TestBytes.family, TestBytes.qualifier, TestBytes.value3));
+    // delete column family
+    txTable.delete(new Delete(TestBytes.row).deleteFamily(TestBytes.family2));
+
+    // verify changed values
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier),
+              null);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2),
+              TestBytes.value2);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier),
+              TestBytes.value3);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier),
+              null);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier2),
+              null);
+
+    // run a scan with VisibilityLevel.ALL, this should return all raw changes by this transaction,
+    // and the raw change by prior transaction
+    //noinspection ConstantConditions
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_ALL);
+    List<KeyValue> expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier, txWp2, KeyValue.Type.DeleteColumn),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier, txWp1, TestBytes.value),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp2, TestBytes.value2),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp1, TestBytes.value2),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp0, KeyValue.Type.DeleteColumn),
+      new KeyValue(TestBytes.row, TestBytes.family2, null, txWp2, KeyValue.Type.DeleteFamily),
+      new KeyValue(TestBytes.row, TestBytes.family2, TestBytes.qualifier, txWp1, TestBytes.value),
+      new KeyValue(TestBytes.row, TestBytes.family2, TestBytes.qualifier2, txWp1, TestBytes.value2),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp2, TestBytes.value3),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp1, TestBytes.value)
+    );
+    verifyScan(txTable, new Scan(), expected);
+
+    // verify a Get is also able to return all snapshot versions
+    Get get = new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier);
+    Cell[] cells = getRow(txTable, get);
+    Assert.assertEquals(2, cells.length);
+    Assert.assertTrue(CellUtil.isDelete(cells[0]));
+    Assert.assertArrayEquals(TestBytes.value, CellUtil.cloneValue(cells[1]));
+
+    get = new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2);
+    cells = getRow(txTable, get);
+    Assert.assertEquals(3, cells.length);
+    Assert.assertArrayEquals(TestBytes.value2, CellUtil.cloneValue(cells[0]));
+    Assert.assertArrayEquals(TestBytes.value2, CellUtil.cloneValue(cells[1]));
+    Assert.assertTrue(CellUtil.isDeleteColumns(cells[2]));
+
+    verifyRows(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier),
+               ImmutableList.of(TestBytes.value3, TestBytes.value));
+
+    get = new Get(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier);
+    cells = getRow(txTable, get);
+    Assert.assertEquals(2, cells.length);
+    Assert.assertTrue(CellUtil.isDelete(cells[0]));
+    Assert.assertArrayEquals(TestBytes.value, CellUtil.cloneValue(cells[1]));
+
+    get = new Get(TestBytes.row).addColumn(TestBytes.family2, TestBytes.qualifier2);
+    cells = getRow(txTable, get);
+    Assert.assertEquals(2, cells.length);
+    Assert.assertTrue(CellUtil.isDelete(cells[0]));
+    Assert.assertArrayEquals(TestBytes.value2, CellUtil.cloneValue(cells[1]));
+
+    // Verify VisibilityLevel.SNAPSHOT
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT);
+    expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp2, TestBytes.value2),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp2, TestBytes.value3)
+    );
+    verifyScan(txTable, new Scan(), expected);
+
+    // Verify VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT);
+    expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier, txWp1, TestBytes.value),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp1, TestBytes.value2),
+      new KeyValue(TestBytes.row, TestBytes.family2, TestBytes.qualifier, txWp1, TestBytes.value),
+      new KeyValue(TestBytes.row, TestBytes.family2, TestBytes.qualifier2, txWp1, TestBytes.value2),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp1, TestBytes.value)
+    );
+    verifyScan(txTable, new Scan(), expected);
+    txContext.finish();
+
+    // finally verify values once more after commit, this time we should get only committed raw values for
+    // all visibility levels
+    txContext.start();
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_ALL);
+    expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier, txWp2, KeyValue.Type.DeleteColumn),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp2, TestBytes.value2),
+      new KeyValue(TestBytes.row, TestBytes.family2, null, txWp2, KeyValue.Type.DeleteFamily),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp2, TestBytes.value3)
+    );
+    verifyScan(txTable, new Scan(), expected);
+
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT);
+    expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp2, TestBytes.value2),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp2, TestBytes.value3)
+    );
+    verifyScan(txTable, new Scan(), expected);
+
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT);
+    expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp2, TestBytes.value2),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp2, TestBytes.value3)
+    );
+    verifyScan(txTable, new Scan(), expected);
+
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier),
+              null);
+    verifyRow(txTable, new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2),
+              TestBytes.value2);
+    verifyRow(txTable, new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier),
+              TestBytes.value3);
+    txContext.finish();
+
+    // Test with regular HBase deletes in pre-existing data
+    long now = System.currentTimeMillis();
+    Delete deleteColumn = new Delete(TestBytes.row3).deleteColumn(TestBytes.family, TestBytes.qualifier, now - 1);
+    // to prevent Tephra from replacing delete with delete marker
+    deleteColumn.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
+    nonTxTable.delete(deleteColumn);
+    Delete deleteFamily = new Delete(TestBytes.row3).deleteFamily(TestBytes.family2, now);
+    // to prevent Tephra from replacing delete with delete marker
+    deleteFamily.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
+    nonTxTable.delete(deleteFamily);
+    nonTxTable.flushCommits();
+
+    txContext.start();
+    txContext.getCurrentTransaction().setVisibility(Transaction.VisibilityLevel.SNAPSHOT_ALL);
+    expected = ImmutableList.of(
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier, txWp2, KeyValue.Type.DeleteColumn),
+      new KeyValue(TestBytes.row, TestBytes.family, TestBytes.qualifier2, txWp2, TestBytes.value2),
+      new KeyValue(TestBytes.row, TestBytes.family2, null, txWp2, KeyValue.Type.DeleteFamily),
+      new KeyValue(TestBytes.row2, TestBytes.family, TestBytes.qualifier, txWp2, TestBytes.value3),
+      new KeyValue(TestBytes.row3, TestBytes.family, TestBytes.qualifier, now - 1, KeyValue.Type.Delete),
+      new KeyValue(TestBytes.row3, TestBytes.family2, null, now, KeyValue.Type.DeleteFamily)
+    );
+    // test scan
+    Scan scan = new Scan();
+    scan.setRaw(true);
+    verifyScan(txTable, scan, expected);
+    txContext.finish();
+  }
+
+  @Test
+  public void testFilters() throws Exception {
+    // Add some values to table
+    transactionContext.start();
+    Put put = new Put(TestBytes.row);
+    byte[] val1 = Bytes.toBytes(1L);
+    put.add(TestBytes.family, TestBytes.qualifier, val1);
+    transactionAwareHTable.put(put);
+    put = new Put(TestBytes.row2);
+    byte[] val2 = Bytes.toBytes(2L);
+    put.add(TestBytes.family, TestBytes.qualifier, val2);
+    transactionAwareHTable.put(put);
+    put = new Put(TestBytes.row3);
+    byte[] val3 = Bytes.toBytes(3L);
+    put.add(TestBytes.family, TestBytes.qualifier, val3);
+    transactionAwareHTable.put(put);
+    put = new Put(TestBytes.row4);
+    byte[] val4 = Bytes.toBytes(4L);
+    put.add(TestBytes.family, TestBytes.qualifier, val4);
+    transactionAwareHTable.put(put);
+    transactionContext.finish();
+
+    // Delete cell with value 2
+    transactionContext.start();
+    Delete delete = new Delete(TestBytes.row2);
+    delete.addColumn(TestBytes.family, TestBytes.qualifier);
+    transactionAwareHTable.delete(delete);
+    transactionContext.finish();
+
+    // Scan for values less than 4, should get only values 1 and 3
+    transactionContext.start();
+    Scan scan = new Scan(TestBytes.row, new ValueFilter(CompareFilter.CompareOp.LESS, new LongComparator(4)));
+    try (ResultScanner scanner = transactionAwareHTable.getScanner(scan)) {
+      Result result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row, result.getRow());
+      assertArrayEquals(val1, result.getValue(TestBytes.family, TestBytes.qualifier));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row3, result.getRow());
+      assertArrayEquals(val3, result.getValue(TestBytes.family, TestBytes.qualifier));
+      result = scanner.next();
+      assertNull(result);
+    }
+    transactionContext.finish();
+
+    // Run a Get with a filter for less than 10 on row4, should get value 4
+    transactionContext.start();
+    Get get = new Get(TestBytes.row4);
+    get.setFilter(new ValueFilter(CompareFilter.CompareOp.LESS, new LongComparator(10)));
+    Result result = transactionAwareHTable.get(get);
+    assertFalse(result.isEmpty());
+    assertArrayEquals(val4, result.getValue(TestBytes.family, TestBytes.qualifier));
+    transactionContext.finish();
+
+    // Change value of row4 to 40
+    transactionContext.start();
+    put = new Put(TestBytes.row4);
+    byte[] val40 = Bytes.toBytes(40L);
+    put.add(TestBytes.family, TestBytes.qualifier, val40);
+    transactionAwareHTable.put(put);
+    transactionContext.finish();
+
+    // Scan for values less than 10, should get only values 1 and 3
+    transactionContext.start();
+    scan = new Scan(TestBytes.row, new ValueFilter(CompareFilter.CompareOp.LESS, new LongComparator(10)));
+    try (ResultScanner scanner = transactionAwareHTable.getScanner(scan)) {
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row, result.getRow());
+      assertArrayEquals(val1, result.getValue(TestBytes.family, TestBytes.qualifier));
+      result = scanner.next();
+      assertNotNull(result);
+      assertArrayEquals(TestBytes.row3, result.getRow());
+      assertArrayEquals(val3, result.getValue(TestBytes.family, TestBytes.qualifier));
+      result = scanner.next();
+      assertNull(result);
+    }
+    transactionContext.finish();
+
+    // Run the Get again with a filter for less than 10 on row4, this time should not get any results
+    transactionContext.start();
+    result = transactionAwareHTable.get(get);
+    assertTrue(result.isEmpty());
+    transactionContext.finish();
+  }
+
+  @Test
+  public void testTxLifetime() throws Exception {
+    // Add some initial values
+    transactionContext.start();
+    Put put = new Put(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+    transactionAwareHTable.put(put);
+    put = new Put(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2, TestBytes.value);
+    transactionAwareHTable.put(put);
+    transactionContext.finish();
+
+    // Simulate writing with a transaction past its max lifetime
+    transactionContext.start();
+    Transaction currentTx = transactionContext.getCurrentTransaction();
+    Assert.assertNotNull(currentTx);
+
+    // Create a transaction that is past the max lifetime
+    long txMaxLifetimeMillis = TimeUnit.SECONDS.toMillis(conf.getInt(TxConstants.Manager.CFG_TX_MAX_LIFETIME,
+                                                                     TxConstants.Manager.DEFAULT_TX_MAX_LIFETIME));
+    long oldTxId = currentTx.getTransactionId() - ((txMaxLifetimeMillis + 10000) * TxConstants.MAX_TX_PER_MS);
+    Transaction oldTx = new Transaction(currentTx.getReadPointer(), oldTxId,
+                                        currentTx.getInvalids(), currentTx.getInProgress(),
+                                        currentTx.getFirstShortInProgress());
+    transactionAwareHTable.updateTx(oldTx);
+    // Put with the old transaction should fail
+    put = new Put(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+    try {
+      transactionAwareHTable.put(put);
+      Assert.fail("Excepted exception with old transaction!");
+    } catch (IOException e) {
+      // Expected exception
+    }
+
+    // Delete with the old transaction should also fail
+    Delete delete = new Delete(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier);
+    try {
+      transactionAwareHTable.delete(delete);
+      Assert.fail("Excepted exception with old transaction!");
+    } catch (IOException e) {
+      // Expected exception
+    }
+
+    // Now update the table to use the current transaction
+    transactionAwareHTable.updateTx(currentTx);
+    put = new Put(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier2, TestBytes.value);
+    transactionAwareHTable.put(put);
+    delete = new Delete(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2);
+    transactionAwareHTable.delete(delete);
+
+    // Verify values with the same transaction since we cannot commit the old transaction
+    verifyRow(transactionAwareHTable,
+              new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier), TestBytes.value);
+    verifyRow(transactionAwareHTable,
+              new Get(TestBytes.row).addColumn(TestBytes.family, TestBytes.qualifier2), null);
+    verifyRow(transactionAwareHTable,
+              new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier), null);
+    verifyRow(transactionAwareHTable,
+              new Get(TestBytes.row2).addColumn(TestBytes.family, TestBytes.qualifier2), TestBytes.value);
+    transactionContext.finish();
+  }
+
+  /**
+   * Tests that transaction co-processor works with older clients
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testOlderClientOperations() throws Exception {
+    // Use old HTable to test
+    TransactionAwareHTable oldTxAware = new OldTransactionAwareHTable(hTable);
+    transactionContext.addTransactionAware(oldTxAware);
+
+    transactionContext.start();
+    Put put = new Put(TestBytes.row);
+    put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value);
+    oldTxAware.put(put);
+    transactionContext.finish();
+
+    transactionContext.start();
+    long txId = transactionContext.getCurrentTransaction().getTransactionId();
+    put = new Put(TestBytes.row);
+    put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value2);
+    oldTxAware.put(put);
+    // Invalidate the second Put
+    TransactionSystemClient txClient = new InMemoryTxSystemClient(txManager);
+    txClient.invalidate(txId);
+
+    transactionContext.start();
+    put = new Put(TestBytes.row);
+    put.add(TestBytes.family, TestBytes.qualifier, TestBytes.value3);
+    oldTxAware.put(put);
+    // Abort the third Put
+    transactionContext.abort();
+
+    // Get should now return the first value
+    transactionContext.start();
+    Result result = oldTxAware.get(new Get(TestBytes.row));
+    transactionContext.finish();
+
+    byte[] value = result.getValue(TestBytes.family, TestBytes.qualifier);
+    assertArrayEquals(TestBytes.value, value);
+  }
+
+  /**
+   * Represents older transaction clients
+   */
+  private static class OldTransactionAwareHTable extends TransactionAwareHTable {
+    public OldTransactionAwareHTable(HTableInterface hTable) {
+      super(hTable);
+    }
+
+    @Override
+    public void addToOperation(OperationWithAttributes op, Transaction tx) throws IOException {
+      op.setAttribute(TxConstants.OLD_TX_OPERATION_ATTRIBUTE_KEY, txCodec.encode(tx));
+    }
+
+    @Override
+    protected void makeRollbackOperation(Delete delete) {
+      delete.setAttribute(TxConstants.OLD_TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]);
+    }
+  }
+}