You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tephra.apache.org by po...@apache.org on 2016/09/08 08:10:00 UTC

[1/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Repository: incubator-tephra
Updated Branches:
  refs/heads/master ae574caf7 -> 2246abffb


http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java b/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
deleted file mode 100644
index 4a694eb..0000000
--- a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
+++ /dev/null
@@ -1,622 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.TransactionType;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.coprocessor.TransactionStateCache;
-import org.apache.tephra.coprocessor.TransactionStateCacheSupplier;
-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
-    TransactionSnapshot txSnapshot = TransactionSnapshot.copyFrom(
-        System.currentTimeMillis(), V[6] - 1, V[7], invalidSet,
-        // this will set visibility upper bound to V[6]
-        Maps.newTreeMap(ImmutableSortedMap.of(V[6], new TransactionManager.InProgressTx(V[6] - 1, Long.MAX_VALUE,
-                                                                                        TransactionType.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});
-    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/2246abff/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java b/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
deleted file mode 100644
index d976085..0000000
--- a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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;
-      }
-    }
-  }
-}



[6/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Posted by po...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
new file mode 100644
index 0000000..d8664f4
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/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/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
new file mode 100644
index 0000000..0ca9f9c
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/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/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
new file mode 100644
index 0000000..14941b3
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
@@ -0,0 +1,358 @@
+/*
+ * 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.base.Supplier;
+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.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+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.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+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.TransactionStateCache;
+import org.apache.tephra.coprocessor.TransactionStateCacheSupplier;
+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;
+
+/**
+ * {@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 TransactionStateCache cache;
+  private final TransactionCodec txCodec;
+  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;
+      Supplier<TransactionStateCache> 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 = env.getConfiguration().getBoolean(TxConstants.ALLOW_EMPTY_VALUES_KEY,
+                                                                TxConstants.ALLOW_EMPTY_VALUES_DEFAULT);
+      this.readNonTxnData = Boolean.valueOf(tableDesc.getValue(TxConstants.READ_NON_TX_DATA));
+      if (readNonTxnData) {
+        LOG.info("Reading pre-existing data enabled for table " + tableDesc.getNameAsString());
+      }
+    }
+  }
+
+  protected Supplier<TransactionStateCache> getTransactionStateCacheSupplier(RegionCoprocessorEnvironment env) {
+    return new TransactionStateCacheSupplier(env.getConfiguration());
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    // nothing to do
+  }
+
+  @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 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;
+    }
+
+    // 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 {
+        int cellSize = familyCells.size();
+        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 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 InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s,
+      CompactionRequest request)
+      throws IOException {
+    return createStoreScanner(c.getEnvironment(), "compaction", cache.getLatestState(), store, scanners,
+                              scanType, earliestPutTs);
+  }
+
+  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;
+  }
+
+  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);
+  }
+
+  /**
+   * 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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
new file mode 100644
index 0000000..a258972
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
@@ -0,0 +1,308 @@
+/*
+ * 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.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.1+ 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;
+
+    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();
+    }
+
+    public boolean isDeleted(Cell cell) {
+      return cell.getTimestamp() <= familyDeleteTs;
+    }
+
+    public void reset() {
+      this.familyDeleteTs = 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java
new file mode 100644
index 0000000..cac80ec
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.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.1 and HBase 1.2 versions specific behavior.
+ */
+public class HBase11ConfigurationProviderTest extends AbstractConfigurationProviderTest {
+  @Override
+  protected Collection<HBaseVersion.Version> getExpectedVersions() {
+    return ImmutableList.of(HBaseVersion.Version.HBASE_11, HBaseVersion.Version.HBASE_12);
+  }
+}



[4/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Posted by po...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
new file mode 100644
index 0000000..4a694eb
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java
@@ -0,0 +1,622 @@
+/*
+ * 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.TransactionType;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.coprocessor.TransactionStateCache;
+import org.apache.tephra.coprocessor.TransactionStateCacheSupplier;
+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
+    TransactionSnapshot txSnapshot = TransactionSnapshot.copyFrom(
+        System.currentTimeMillis(), V[6] - 1, V[7], invalidSet,
+        // this will set visibility upper bound to V[6]
+        Maps.newTreeMap(ImmutableSortedMap.of(V[6], new TransactionManager.InProgressTx(V[6] - 1, Long.MAX_VALUE,
+                                                                                        TransactionType.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});
+    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/2246abff/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java
new file mode 100644
index 0000000..d976085
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/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/2246abff/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.1/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.1/pom.xml b/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.1/pom.xml
new file mode 100644
index 0000000..537e237
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.1/pom.xml
@@ -0,0 +1,117 @@
+<?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-hbase-compat-1.1-base</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-hbase-compat-1.1</artifactId>
+  <name>Apache Tephra HBase 1.1 Compatibility</name>
+
+  <properties>
+    <hadoop.version>2.5.1</hadoop.version>
+    <hbase11.version>1.1.1</hbase11.version>
+  </properties>
+
+  <build>
+    <sourceDirectory>../src/main/java</sourceDirectory>
+    <testSourceDirectory>../src/test/java</testSourceDirectory>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase11.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>${hbase11.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>${hbase11.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase11.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>${hbase11.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>${hbase11.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/2246abff/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2-cdh/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2-cdh/pom.xml b/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2-cdh/pom.xml
new file mode 100644
index 0000000..ad84a05
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2-cdh/pom.xml
@@ -0,0 +1,125 @@
+<?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-hbase-compat-1.1-base</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-hbase-compat-1.2-cdh</artifactId>
+  <name>Apache Tephra HBase 1.2 Compatibility for CDH</name>
+
+  <properties>
+    <!-- Hadoop version supported by CDH -->
+    <hadoop.version>2.6.0-cdh5.7.0</hadoop.version>
+    <hbase12cdh.version>1.2.0-cdh5.7.0</hbase12cdh.version>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>cloudera</id>
+      <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+    </repository>
+  </repositories>
+
+  <build>
+    <sourceDirectory>../src/main/java</sourceDirectory>
+    <testSourceDirectory>../src/test/java</testSourceDirectory>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase12cdh.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>${hbase12cdh.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>${hbase12cdh.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase12cdh.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>${hbase12cdh.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>${hbase12cdh.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/2246abff/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2/pom.xml b/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2/pom.xml
new file mode 100644
index 0000000..18b26dd
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/tephra-hbase-compat-1.2/pom.xml
@@ -0,0 +1,118 @@
+<?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-hbase-compat-1.1-base</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-hbase-compat-1.2</artifactId>
+  <name>Apache Tephra HBase 1.2 Compatibility</name>
+
+  <properties>
+    <hadoop.version>2.5.1</hadoop.version>
+    <hbase12.version>1.2.0</hbase12.version>
+
+  </properties>
+
+  <build>
+    <sourceDirectory>../src/main/java</sourceDirectory>
+    <testSourceDirectory>../src/test/java</testSourceDirectory>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase12.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>${hbase12.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>${hbase12.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase12.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>${hbase12.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>${hbase12.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/2246abff/tephra-hbase-compat-1.1/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/pom.xml b/tephra-hbase-compat-1.1/pom.xml
deleted file mode 100644
index 543ee7f..0000000
--- a/tephra-hbase-compat-1.1/pom.xml
+++ /dev/null
@@ -1,150 +0,0 @@
-<?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.9.0-incubating-SNAPSHOT</version>
-  </parent>
-  <modelVersion>4.0.0</modelVersion>
-
-  <artifactId>tephra-hbase-compat-1.1</artifactId>
-  <name>Apache Tephra HBase 1.1 Compatibility</name>
-
-  <properties>
-    <!-- HBase 1.1 only supports Hadoop 2.4 or newer -->
-    <hadoop.version>2.4.0</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>${hbase11.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>${hbase11.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>${hbase11.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-server</artifactId>
-      <version>${hbase11.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>${hbase11.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-testing-util</artifactId>
-      <version>${hbase11.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>
-
-  <profiles>
-    <profile>
-      <id>hbase1.2</id>
-      <properties>
-        <hbase11.version>1.2.1</hbase11.version>
-      </properties>
-    </profile>
-  </profiles>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java
deleted file mode 100644
index 7ab7a18..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.1 version of {@link ConfigurationProvider}.
- */
-public class HBase11ConfigurationProvider 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/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
deleted file mode 100644
index 8bf8768..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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();
-  }
-}


[5/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Posted by po...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
new file mode 100644
index 0000000..de1fa6b
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
@@ -0,0 +1,1606 @@
+/*
+ * 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.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+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.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.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.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.TransactionStateStorage;
+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 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 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 {
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionAwareHTableTest.class);
+
+  private static HBaseTestingUtility testUtil;
+  private static HBaseAdmin hBaseAdmin;
+  private static TransactionStateStorage txStateStorage;
+  private static TransactionManager txManager;
+  private static Configuration conf;
+  private TransactionContext transactionContext;
+  private TransactionAwareHTable transactionAwareHTable;
+  private HTable hTable;
+
+  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();
+
+    // 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 InMemoryTransactionStateStorage();
+    txManager = new TransactionManager(conf, txStateStorage, new TxMetricsCollector());
+    txManager.startAndWait();
+  }
+
+  @AfterClass
+  public static void shutdownAfterClass() throws Exception {
+    testUtil.shutdownMiniCluster();
+    hBaseAdmin.close();
+  }
+
+  @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);
+  }
+
+  private HTable createTable(byte[] tableName, byte[][] columnFamilies) throws Exception {
+    return createTable(tableName, columnFamilies, false, Collections.<String>emptyList());
+  }
+
+  private 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; 
+    desc.addCoprocessor(TransactionProcessor.class.getName(), null, priority, null);
+    // 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);
+   }
+
+  /**
+   * 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(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 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.<String>emptyList()));
+    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.<String>emptyList());
+    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();
+  }
+
+  /**
+   * 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]);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java b/tephra-hbase-compat-1.1-base/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
new file mode 100644
index 0000000..428d3b0
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/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.1 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;
+    }
+  }
+}


[3/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Posted by po...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
deleted file mode 100644
index bb7afff..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
+++ /dev/null
@@ -1,678 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.1. 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 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());
-    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/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
deleted file mode 100644
index d8664f4..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
deleted file mode 100644
index 0ca9f9c..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
deleted file mode 100644
index 14941b3..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java
+++ /dev/null
@@ -1,358 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.base.Supplier;
-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.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-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.RegionScanner;
-import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.Store;
-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.TransactionStateCache;
-import org.apache.tephra.coprocessor.TransactionStateCacheSupplier;
-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;
-
-/**
- * {@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 TransactionStateCache cache;
-  private final TransactionCodec txCodec;
-  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;
-      Supplier<TransactionStateCache> 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 = env.getConfiguration().getBoolean(TxConstants.ALLOW_EMPTY_VALUES_KEY,
-                                                                TxConstants.ALLOW_EMPTY_VALUES_DEFAULT);
-      this.readNonTxnData = Boolean.valueOf(tableDesc.getValue(TxConstants.READ_NON_TX_DATA));
-      if (readNonTxnData) {
-        LOG.info("Reading pre-existing data enabled for table " + tableDesc.getNameAsString());
-      }
-    }
-  }
-
-  protected Supplier<TransactionStateCache> getTransactionStateCacheSupplier(RegionCoprocessorEnvironment env) {
-    return new TransactionStateCacheSupplier(env.getConfiguration());
-  }
-
-  @Override
-  public void stop(CoprocessorEnvironment e) throws IOException {
-    // nothing to do
-  }
-
-  @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 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;
-    }
-
-    // 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 {
-        int cellSize = familyCells.size();
-        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 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 InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s,
-      CompactionRequest request)
-      throws IOException {
-    return createStoreScanner(c.getEnvironment(), "compaction", cache.getLatestState(), store, scanners,
-                              scanType, earliestPutTs);
-  }
-
-  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;
-  }
-
-  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);
-  }
-
-  /**
-   * 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);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java b/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
deleted file mode 100644
index a258972..0000000
--- a/tephra-hbase-compat-1.1/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java
+++ /dev/null
@@ -1,308 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.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.1+ 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;
-
-    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();
-    }
-
-    public boolean isDeleted(Cell cell) {
-      return cell.getTimestamp() <= familyDeleteTs;
-    }
-
-    public void reset() {
-      this.familyDeleteTs = 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java b/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java
deleted file mode 100644
index cac80ec..0000000
--- a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/HBase11ConfigurationProviderTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.1 and HBase 1.2 versions specific behavior.
- */
-public class HBase11ConfigurationProviderTest extends AbstractConfigurationProviderTest {
-  @Override
-  protected Collection<HBaseVersion.Version> getExpectedVersions() {
-    return ImmutableList.of(HBaseVersion.Version.HBASE_11, HBaseVersion.Version.HBASE_12);
-  }
-}


[7/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Posted by po...@apache.org.
TEPHRA-176,TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

This closes #4 and closes #1

Signed-off-by: poorna <po...@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/2246abff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tephra/tree/2246abff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tephra/diff/2246abff

Branch: refs/heads/master
Commit: 2246abffb75af10dc8284b559ecfe781d4bfe8ff
Parents: ae574ca
Author: shankar <sh...@cask.co>
Authored: Fri Sep 2 13:39:26 2016 -0700
Committer: poorna <po...@apache.org>
Committed: Thu Sep 8 01:05:44 2016 -0700

----------------------------------------------------------------------
 pom.xml                                         |    4 +-
 tephra-examples/cdh-5.7/pom.xml                 |  128 ++
 tephra-examples/cdh-5.8/pom.xml                 |  130 ++
 tephra-examples/hbase-0.96/pom.xml              |  122 ++
 tephra-examples/hbase-0.98/pom.xml              |  123 ++
 tephra-examples/hbase-1.0-cdh/pom.xml           |  129 ++
 tephra-examples/hbase-1.0/pom.xml               |  122 ++
 tephra-examples/hbase-1.1/pom.xml               |  122 ++
 tephra-examples/hbase-1.2/pom.xml               |  122 ++
 tephra-examples/pom.xml                         |   84 +-
 tephra-hbase-compat-1.0-cdh/pom.xml             |    6 +
 tephra-hbase-compat-1.0/pom.xml                 |    6 +
 tephra-hbase-compat-1.1-base/pom.xml            |   77 +
 .../hbase/HBase11ConfigurationProvider.java     |   38 +
 .../tephra/hbase/SecondaryIndexTable.java       |  178 ++
 .../tephra/hbase/TransactionAwareHTable.java    |  678 ++++++++
 .../hbase/coprocessor/CellSkipFilter.java       |  138 ++
 .../hbase/coprocessor/TransactionFilters.java   |   62 +
 .../hbase/coprocessor/TransactionProcessor.java |  358 ++++
 .../TransactionVisibilityFilter.java            |  308 ++++
 .../hbase/HBase11ConfigurationProviderTest.java |   35 +
 .../hbase/TransactionAwareHTableTest.java       | 1606 ++++++++++++++++++
 .../hbase/coprocessor/CellSkipFilterTest.java   |  123 ++
 .../coprocessor/TransactionProcessorTest.java   |  622 +++++++
 .../TransactionVisibilityFilterTest.java        |  374 ++++
 .../tephra-hbase-compat-1.1/pom.xml             |  117 ++
 .../tephra-hbase-compat-1.2-cdh/pom.xml         |  125 ++
 .../tephra-hbase-compat-1.2/pom.xml             |  118 ++
 tephra-hbase-compat-1.1/pom.xml                 |  150 --
 .../hbase/HBase11ConfigurationProvider.java     |   38 -
 .../tephra/hbase/SecondaryIndexTable.java       |  178 --
 .../tephra/hbase/TransactionAwareHTable.java    |  678 --------
 .../hbase/coprocessor/CellSkipFilter.java       |  138 --
 .../hbase/coprocessor/TransactionFilters.java   |   62 -
 .../hbase/coprocessor/TransactionProcessor.java |  358 ----
 .../TransactionVisibilityFilter.java            |  308 ----
 .../hbase/HBase11ConfigurationProviderTest.java |   35 -
 .../hbase/TransactionAwareHTableTest.java       | 1606 ------------------
 .../hbase/coprocessor/CellSkipFilterTest.java   |  123 --
 .../coprocessor/TransactionProcessorTest.java   |  622 -------
 .../TransactionVisibilityFilterTest.java        |  374 ----
 41 files changed, 5983 insertions(+), 4742 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ba992b7..5e1a7d3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -156,7 +156,7 @@
     <module>tephra-hbase-compat-0.98</module>
     <module>tephra-hbase-compat-1.0</module>
     <module>tephra-hbase-compat-1.0-cdh</module>
-    <module>tephra-hbase-compat-1.1</module>
+    <module>tephra-hbase-compat-1.1-base</module>
     <module>tephra-examples</module>
     <module>tephra-distribution</module>
   </modules>
@@ -210,6 +210,8 @@
     <hbase10.version>1.0.1.1</hbase10.version>
     <hbase10cdh.version>1.0.0-cdh5.4.2</hbase10cdh.version>
     <hbase11.version>1.1.1</hbase11.version>
+    <hbase12cdh.version>1.2.0-cdh5.7.0</hbase12cdh.version>
+    <hbase12.version>1.2.0</hbase12.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/2246abff/tephra-examples/cdh-5.7/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/cdh-5.7/pom.xml b/tephra-examples/cdh-5.7/pom.xml
new file mode 100644
index 0000000..bc3ae96
--- /dev/null
+++ b/tephra-examples/cdh-5.7/pom.xml
@@ -0,0 +1,128 @@
+<?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.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-cdh-5.7</artifactId>
+  <name>Apache Tephra Examples For CDH 5.7</name>
+
+  <repositories>
+    <repository>
+      <id>cloudera</id>
+      <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+    </repository>
+  </repositories>
+
+  <properties>
+    <hadoop.version>2.6.0-cdh5.7.0</hadoop.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.2-cdh</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase12cdh.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>${hbase12cdh.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>${hbase12cdh.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase12cdh.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>${hbase12cdh.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>${hbase12cdh.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/2246abff/tephra-examples/cdh-5.8/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/cdh-5.8/pom.xml b/tephra-examples/cdh-5.8/pom.xml
new file mode 100644
index 0000000..d69355d
--- /dev/null
+++ b/tephra-examples/cdh-5.8/pom.xml
@@ -0,0 +1,130 @@
+<?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>
+    <artifactId>tephra-examples</artifactId>
+    <groupId>org.apache.tephra</groupId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-cdh-5.8</artifactId>
+  <name>Apache Tephra Examples For CDH 5.8</name>
+
+  <repositories>
+    <repository>
+      <id>cloudera</id>
+      <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+    </repository>
+  </repositories>
+
+  <properties>
+    <hadoop.version>2.6.0-cdh5.8.0</hadoop.version>
+    <hbase12cdh.version>1.2.0-cdh5.8.0</hbase12cdh.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.2-cdh</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase12cdh.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>${hbase12cdh.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>${hbase12cdh.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase12cdh.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>${hbase12cdh.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>${hbase12cdh.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/2246abff/tephra-examples/hbase-0.96/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-0.96/pom.xml b/tephra-examples/hbase-0.96/pom.xml
new file mode 100644
index 0000000..9293bc3
--- /dev/null
+++ b/tephra-examples/hbase-0.96/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>
+    <artifactId>tephra-examples</artifactId>
+    <groupId>org.apache.tephra</groupId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-0.96</artifactId>
+  <name>Apache Tephra Examples For Hbase 0.96</name>
+
+  <properties>
+    <hadoop.version>2.3.0</hadoop.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-0.96</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase96.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>${hbase96.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>${hbase96.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase96.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>${hbase96.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>${hbase96.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/2246abff/tephra-examples/hbase-0.98/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-0.98/pom.xml b/tephra-examples/hbase-0.98/pom.xml
new file mode 100644
index 0000000..37ee5a2
--- /dev/null
+++ b/tephra-examples/hbase-0.98/pom.xml
@@ -0,0 +1,123 @@
+<?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>
+    <artifactId>tephra-examples</artifactId>
+    <groupId>org.apache.tephra</groupId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-0.98</artifactId>
+  <name>Apache Tephra Examples For Hbase 0.98</name>
+
+  <properties>
+    <hadoop.version>2.5.0</hadoop.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-0.98</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase98.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>${hbase98.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>${hbase98.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase98.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>${hbase98.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>${hbase98.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/2246abff/tephra-examples/hbase-1.0-cdh/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-1.0-cdh/pom.xml b/tephra-examples/hbase-1.0-cdh/pom.xml
new file mode 100644
index 0000000..6f76d85
--- /dev/null
+++ b/tephra-examples/hbase-1.0-cdh/pom.xml
@@ -0,0 +1,129 @@
+<?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>
+    <artifactId>tephra-examples</artifactId>
+    <groupId>org.apache.tephra</groupId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-1.0-cdh</artifactId>
+  <name>Apache Tephra Examples For Hbase 1.0-cdh</name>
+
+  <properties>
+    <hadoop.version>2.6.0</hadoop.version>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>cloudera</id>
+      <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+    </repository>
+  </repositories>
+
+  <build>
+    <sourceDirectory>../src/main/java</sourceDirectory>
+    <testSourceDirectory>../src/test/java</testSourceDirectory>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tephra</groupId>
+      <artifactId>tephra-hbase-compat-1.0-cdh</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase10cdh.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>${hbase10cdh.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>${hbase10cdh.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase10cdh.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>${hbase10cdh.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>${hbase10cdh.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/2246abff/tephra-examples/hbase-1.0/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-1.0/pom.xml b/tephra-examples/hbase-1.0/pom.xml
new file mode 100644
index 0000000..5e3c619
--- /dev/null
+++ b/tephra-examples/hbase-1.0/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>
+    <artifactId>tephra-examples</artifactId>
+    <groupId>org.apache.tephra</groupId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-1.0</artifactId>
+  <name>Apache Tephra Examples For Hbase 1.0</name>
+
+  <properties>
+    <hadoop.version>2.6.0</hadoop.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.0</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase10.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>${hbase10.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>${hbase10.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase10.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>${hbase10.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>${hbase10.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/2246abff/tephra-examples/hbase-1.1/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-1.1/pom.xml b/tephra-examples/hbase-1.1/pom.xml
new file mode 100644
index 0000000..3395bb9
--- /dev/null
+++ b/tephra-examples/hbase-1.1/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.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-1.1</artifactId>
+  <name>Apache Tephra Examples For Hbase 1.1</name>
+
+  <properties>
+    <hadoop.version>2.5.1</hadoop.version>
+    <hbase11.version>1.1.1</hbase11.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.1</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase11.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>${hbase11.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>${hbase11.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase11.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>${hbase11.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>${hbase11.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/2246abff/tephra-examples/hbase-1.2/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-1.2/pom.xml b/tephra-examples/hbase-1.2/pom.xml
new file mode 100644
index 0000000..d37080d
--- /dev/null
+++ b/tephra-examples/hbase-1.2/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.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-1.2</artifactId>
+  <name>Apache Tephra Examples For HBase 1.2</name>
+
+  <properties>
+    <hadoop.version>2.5.1</hadoop.version>
+    <hbase12.version>1.2.0</hbase12.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.2</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase12.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>${hbase12.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>${hbase12.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase12.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>${hbase12.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>${hbase12.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/2246abff/tephra-examples/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/pom.xml b/tephra-examples/pom.xml
index 6f151cd..07d0db0 100644
--- a/tephra-examples/pom.xml
+++ b/tephra-examples/pom.xml
@@ -28,11 +28,18 @@
   <artifactId>tephra-examples</artifactId>
   <name>Apache Tephra Examples</name>
 
-  <properties>
-    <hadoop.version>2.6.0</hadoop.version>
-    <hbase12.version>1.2.1</hbase12.version>
-  </properties>
-  
+  <packaging>pom</packaging>
+  <modules>
+    <module>hbase-0.96</module>
+    <module>hbase-0.98</module>
+    <module>hbase-1.0</module>
+    <module>hbase-1.0-cdh</module>
+    <module>hbase-1.1</module>
+    <module>hbase-1.2</module>
+    <module>cdh-5.7</module>
+    <module>cdh-5.8</module>
+  </modules>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.tephra</groupId>
@@ -44,53 +51,7 @@
       <artifactId>tephra-core</artifactId>
       <version>${project.version}</version>
     </dependency>
-    <dependency>
-      <groupId>org.apache.tephra</groupId>
-      <artifactId>tephra-hbase-compat-1.1</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-common</artifactId>
-      <version>${hbase12.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>${hbase12.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>${hbase12.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-server</artifactId>
-      <version>${hbase12.version}</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-log4j12</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
+
 
     <!-- Test dependencies -->
     <dependency>
@@ -112,25 +73,6 @@
       <groupId>org.slf4j</groupId>
       <artifactId>jcl-over-slf4j</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-server</artifactId>
-      <version>${hbase12.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-testing-util</artifactId>
-      <version>${hbase12.version}</version>
-      <scope>test</scope>
-      <exclusions>
-        <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/2246abff/tephra-hbase-compat-1.0-cdh/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.0-cdh/pom.xml b/tephra-hbase-compat-1.0-cdh/pom.xml
index cfad3b6..fb383c5 100644
--- a/tephra-hbase-compat-1.0-cdh/pom.xml
+++ b/tephra-hbase-compat-1.0-cdh/pom.xml
@@ -121,6 +121,12 @@
       <version>${hbase10cdh.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>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.0/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.0/pom.xml b/tephra-hbase-compat-1.0/pom.xml
index 6d361bf..9711605 100644
--- a/tephra-hbase-compat-1.0/pom.xml
+++ b/tephra-hbase-compat-1.0/pom.xml
@@ -119,6 +119,12 @@
       <version>${hbase10.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>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/pom.xml b/tephra-hbase-compat-1.1-base/pom.xml
new file mode 100644
index 0000000..22ab214
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/pom.xml
@@ -0,0 +1,77 @@
+<?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.9.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-hbase-compat-1.1-base</artifactId>
+  <name>Apache Tephra HBase 1.1 Compatibility Base</name>
+
+  <packaging>pom</packaging>
+  <modules>
+    <module>tephra-hbase-compat-1.2-cdh</module>
+    <module>tephra-hbase-compat-1.1</module>
+    <module>tephra-hbase-compat-1.2</module>
+  </modules>
+
+  <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>
+
+    <!-- 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>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.java
new file mode 100644
index 0000000..7ab7a18
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/HBase11ConfigurationProvider.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.1 version of {@link ConfigurationProvider}.
+ */
+public class HBase11ConfigurationProvider 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/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
new file mode 100644
index 0000000..8bf8768
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/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/2246abff/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
new file mode 100644
index 0000000..bb7afff
--- /dev/null
+++ b/tephra-hbase-compat-1.1-base/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
@@ -0,0 +1,678 @@
+/*
+ * 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.1. 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 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());
+    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]);
+  }
+}


[2/7] incubator-tephra git commit: TEPHRA-176, TEPHRA-177: Adding maven modules for CDH-5.7, 5.8 support, HBase-1.1 and HBase-1.2 modules

Posted by po...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java b/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
deleted file mode 100644
index de1fa6b..0000000
--- a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java
+++ /dev/null
@@ -1,1606 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-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.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.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.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.TransactionStateStorage;
-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 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 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 {
-  private static final Logger LOG = LoggerFactory.getLogger(TransactionAwareHTableTest.class);
-
-  private static HBaseTestingUtility testUtil;
-  private static HBaseAdmin hBaseAdmin;
-  private static TransactionStateStorage txStateStorage;
-  private static TransactionManager txManager;
-  private static Configuration conf;
-  private TransactionContext transactionContext;
-  private TransactionAwareHTable transactionAwareHTable;
-  private HTable hTable;
-
-  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();
-
-    // 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 InMemoryTransactionStateStorage();
-    txManager = new TransactionManager(conf, txStateStorage, new TxMetricsCollector());
-    txManager.startAndWait();
-  }
-
-  @AfterClass
-  public static void shutdownAfterClass() throws Exception {
-    testUtil.shutdownMiniCluster();
-    hBaseAdmin.close();
-  }
-
-  @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);
-  }
-
-  private HTable createTable(byte[] tableName, byte[][] columnFamilies) throws Exception {
-    return createTable(tableName, columnFamilies, false, Collections.<String>emptyList());
-  }
-
-  private 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; 
-    desc.addCoprocessor(TransactionProcessor.class.getName(), null, priority, null);
-    // 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);
-   }
-
-  /**
-   * 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(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 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.<String>emptyList()));
-    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.<String>emptyList());
-    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();
-  }
-
-  /**
-   * 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]);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/2246abff/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java b/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
deleted file mode 100644
index 428d3b0..0000000
--- a/tephra-hbase-compat-1.1/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      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.1 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;
-    }
-  }
-}