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 2018/03/10 01:56:35 UTC

[6/6] incubator-tephra git commit: TEPHRA-272 Add HBase 2.0 compatibility module

TEPHRA-272 Add HBase 2.0 compatibility module

This closes #67

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/db6ef6d2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tephra/tree/db6ef6d2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tephra/diff/db6ef6d2

Branch: refs/heads/master
Commit: db6ef6d2bbab51a8b48293d60758119f5f9e5e22
Parents: 1a352c1
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 18 17:55:08 2017 +0530
Committer: poorna <po...@apache.org>
Committed: Fri Mar 9 17:55:45 2018 -0800

----------------------------------------------------------------------
 .travis.yml                                     |   17 +-
 pom.xml                                         |    2 +
 .../tephra/util/ConfigurationFactory.java       |    7 +-
 .../org/apache/tephra/util/HBaseVersion.java    |    4 +
 .../util/HBaseVersionSpecificFactory.java       |    4 +
 tephra-examples/hbase-1.4/pom.xml               |    2 +-
 tephra-examples/hbase-2.0/pom.xml               |  170 ++
 .../apache/tephra/examples/BalanceBooks.java    |  341 ++++
 .../apache/tephra/examples/package-info.java    |   40 +
 .../tephra/examples/BalanceBooksTest.java       |  137 ++
 tephra-examples/pom.xml                         |    1 +
 tephra-hbase-compat-1.4/pom.xml                 |   24 +
 tephra-hbase-compat-2.0/pom.xml                 |  175 ++
 .../hbase/HBase20ConfigurationProvider.java     |   38 +
 .../tephra/hbase/SecondaryIndexTable.java       |  185 ++
 .../tephra/hbase/TransactionAwareHTable.java    |  721 +++++++
 .../hbase/coprocessor/CellSkipFilter.java       |  129 ++
 .../coprocessor/FilteredInternalScanner.java    |   82 +
 .../hbase/coprocessor/TransactionFilters.java   |   62 +
 .../hbase/coprocessor/TransactionProcessor.java |  574 ++++++
 .../TransactionVisibilityFilter.java            |  305 +++
 .../tephra/hbase/txprune/CompactionState.java   |  112 ++
 .../tephra/hbase/txprune/DataJanitorState.java  |  536 +++++
 .../txprune/HBaseTransactionPruningPlugin.java  |  408 ++++
 .../txprune/InvalidListPruningDebugTool.java    |  429 ++++
 .../hbase/txprune/PruneUpperBoundWriter.java    |  164 ++
 .../txprune/PruneUpperBoundWriterSupplier.java  |   55 +
 .../tephra/hbase/txprune/TimeRegions.java       |   85 +
 .../tephra/hbase/AbstractHBaseTableTest.java    |  105 +
 .../hbase/HBase20ConfigurationProviderTest.java |   35 +
 .../hbase/TransactionAwareHTableTest.java       | 1861 ++++++++++++++++++
 .../hbase/coprocessor/CellSkipFilterTest.java   |  123 ++
 .../coprocessor/TransactionProcessorTest.java   |  676 +++++++
 .../TransactionVisibilityFilterTest.java        |  373 ++++
 .../hbase/txprune/DataJanitorStateTest.java     |  285 +++
 .../hbase/txprune/InvalidListPruneTest.java     |  526 +++++
 .../txprune/InvalidListPruningDebugTest.java    |  432 ++++
 .../PruneUpperBoundWriterSupplierTest.java      |  122 ++
 .../src/test/resources/logback-test.xml         |   39 +
 39 files changed, 9380 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 59a3f20..f5a06b8 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -21,6 +21,8 @@ language: java
 # Run it on precise, until we figure out how to make jdk7 work (or no longer support jdk7).
 dist: precise
 
+install: true
+
 jdk:
   - oraclejdk7
   - oraclejdk8
@@ -30,10 +32,17 @@ jdk:
 # Note: -DskipCoreTests will active a profile in tephra-core that skips tests
 env:
  - OPTIONS='-am -pl tephra-core'
- - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-0.96,tephra-examples/hbase-0.98'
- - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-1.0,tephra-examples/hbase-1.0-cdh'
- - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-1.1,tephra-examples/hbase-1.2,tephra-examples/cdh-5.7,tephra-examples/cdh-5.8'
- - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-1.3,tephra-examples/hbase-1.4'
+ - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-0.96,tephra-hbase-compat-0.98,tephra-examples/hbase-0.96,tephra-examples/hbase-0.98'
+ - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.0,tephra-hbase-compat-1.0-cdh,tephra-examples/hbase-1.0,tephra-examples/hbase-1.0-cdh'
+ - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.1-base,tephra-examples/hbase-1.1,tephra-examples/hbase-1.2,tephra-examples/cdh-5.7,tephra-examples/cdh-5.8'
+ - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.3,tephra-examples/hbase-1.3'
+ - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.4,tephra-examples/hbase-1.4'
+ - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-2.0,tephra-examples/hbase-2.0'
+
+matrix:
+  exclude:
+  - jdk: oraclejdk7
+    env: OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-2.0,tephra-examples/hbase-2.0'
 
 branches:
   only:

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 072df6b..0975b92 100644
--- a/pom.xml
+++ b/pom.xml
@@ -159,6 +159,7 @@
     <module>tephra-hbase-compat-1.1-base</module>
     <module>tephra-hbase-compat-1.3</module>
     <module>tephra-hbase-compat-1.4</module>
+    <module>tephra-hbase-compat-2.0</module>
     <module>tephra-examples</module>
     <module>tephra-distribution</module>
   </modules>
@@ -217,6 +218,7 @@
     <hbase12.version>1.2.0</hbase12.version>
     <hbase13.version>1.3.1</hbase13.version>
     <hbase14.version>1.4.0</hbase14.version>
+    <hbase2.version>2.0.0-beta-1</hbase2.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/db6ef6d2/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java b/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
index 42b57d3..3b39f76 100644
--- a/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
+++ b/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java
@@ -51,7 +51,12 @@ public class ConfigurationFactory implements Provider<Configuration> {
     protected String getHBase13Classname() {
       return "org.apache.tephra.hbase.HBase13ConfigurationProvider";
     }
-
+    
+    @Override
+    protected String getHBase20Classname() {
+      return "org.apache.tephra.hbase.HBase20ConfigurationProvider";
+    }
+    
     @Override
     protected String getHBase10CDHClassname() {
       return "org.apache.tephra.hbase.HBase10ConfigurationProvider";

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
index 84fee8e..d9bec3b 100644
--- a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
+++ b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java
@@ -39,6 +39,7 @@ public class HBaseVersion {
   private static final String HBASE_12_VERSION = "1.2";
   private static final String HBASE_13_VERSION = "1.3";
   private static final String HBASE_14_VERSION = "1.4";
+  private static final String HBASE_20_VERSION = "2.0";
   private static final String CDH_CLASSIFIER = "cdh";
 
   private static final Logger LOG = LoggerFactory.getLogger(HBaseVersion.class);
@@ -56,6 +57,7 @@ public class HBaseVersion {
     HBASE_12("1.2"),
     HBASE_13("1.3"),
     HBASE_14("1.4"),
+    HBASE_20("2.0"),
     UNKNOWN("unknown");
 
     final String majorVersion;
@@ -97,6 +99,8 @@ public class HBaseVersion {
         currentVersion = Version.HBASE_13;
       } else if (versionString.startsWith(HBASE_14_VERSION)) {
         currentVersion = Version.HBASE_14;
+      } else if (versionString.startsWith(HBASE_20_VERSION)) {
+          currentVersion = Version.HBASE_20;
       } else {
         currentVersion = Version.UNKNOWN;
       }

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
----------------------------------------------------------------------
diff --git a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
index 685763d..8beef1f 100644
--- a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
+++ b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java
@@ -58,6 +58,9 @@ public abstract class HBaseVersionSpecificFactory<T> implements Provider<T> {
         case HBASE_14:
           instance = createInstance(getHBase14Classname());
           break;
+        case HBASE_20:
+            instance = createInstance(getHBase20Classname());
+            break;
         case UNKNOWN:
         default:
           throw new ProvisionException("Unknown HBase version: " + HBaseVersion.getVersionString());
@@ -80,4 +83,5 @@ public abstract class HBaseVersionSpecificFactory<T> implements Provider<T> {
   protected abstract String getHBase11Classname();
   protected abstract String getHBase13Classname();
   protected abstract String getHBase14Classname();
+  protected abstract String getHBase20Classname();
 }

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-examples/hbase-1.4/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-1.4/pom.xml b/tephra-examples/hbase-1.4/pom.xml
index 217631c..a9fce19 100644
--- a/tephra-examples/hbase-1.4/pom.xml
+++ b/tephra-examples/hbase-1.4/pom.xml
@@ -41,7 +41,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.tephra</groupId>
-      <artifactId>tephra-hbase-compat-1.3</artifactId>
+      <artifactId>tephra-hbase-compat-1.4</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-examples/hbase-2.0/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-2.0/pom.xml b/tephra-examples/hbase-2.0/pom.xml
new file mode 100644
index 0000000..e3fc693
--- /dev/null
+++ b/tephra-examples/hbase-2.0/pom.xml
@@ -0,0 +1,170 @@
+<?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.14.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-examples-hbase-2.0</artifactId>
+  <name>Apache Tephra Examples For HBase 2.0</name>
+
+  <properties>
+    <hadoop.version>2.7.4</hadoop.version>
+  </properties>
+
+  <build>
+    <sourceDirectory>src/main/java</sourceDirectory>
+    <testSourceDirectory>src/test/java</testSourceDirectory>
+      <plugins>
+        <!-- Compiler -->
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-compiler-plugin</artifactId>
+          <version>3.1</version>
+          <configuration>
+            <source>1.8</source>
+            <target>1.8</target>
+          </configuration>
+        </plugin>
+      </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tephra</groupId>
+      <artifactId>tephra-hbase-compat-2.0</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${hbase2.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>${hbase2.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>${hbase2.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase2.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>${hbase2.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.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>1.6.4</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <version>1.2.17</version>
+      <scope>test</scope>
+    </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-testing-util</artifactId>
+      <version>${hbase2.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>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <version>${hbase2.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>asm</groupId>
+          <artifactId>asm</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-http</artifactId>
+      <version>${hbase2.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java
new file mode 100644
index 0000000..b970598
--- /dev/null
+++ b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java
@@ -0,0 +1,341 @@
+/*
+ * 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.examples;
+
+import com.google.common.io.Closeables;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tephra.TransactionConflictException;
+import org.apache.tephra.TransactionContext;
+import org.apache.tephra.TransactionFailureException;
+import org.apache.tephra.TransactionSystemClient;
+import org.apache.tephra.distributed.TransactionServiceClient;
+import org.apache.tephra.hbase.TransactionAwareHTable;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+import org.apache.tephra.runtime.ConfigModule;
+import org.apache.tephra.runtime.DiscoveryModules;
+import org.apache.tephra.runtime.TransactionClientModule;
+import org.apache.tephra.runtime.TransactionModules;
+import org.apache.tephra.runtime.ZKModule;
+import org.apache.tephra.util.ConfigurationFactory;
+import org.apache.twill.zookeeper.ZKClientService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Simple example application that launches a number of concurrent clients, one per "account".  Each client attempts to
+ * make withdrawals from other clients, and deposit the same amount to its own account in a single transaction.
+ * Since this means the client will be updating both its own row and the withdrawee's row, this will naturally lead to
+ * transaction conflicts.  All clients will run for a specified number of iterations.  When the processing is complete,
+ * the total sum of all rows should be zero, if transactional integrity was maintained.
+ *
+ * <p>
+ *   You can run the BalanceBooks application with the following command:
+ *   <pre>
+ *     ./bin/tephra run org.apache.tephra.examples.BalanceBooks [num clients] [num iterations]
+ *   </pre>
+ *   where <code>[num clients]</code> is the number of concurrent client threads to use, and
+ *   <code>[num iterations]</code> is the number of "transfer" operations to perform per client thread.
+ * </p>
+ */
+public class BalanceBooks implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(BalanceBooks.class);
+
+  private static final int MAX_AMOUNT = 100;
+  private static final byte[] TABLE = Bytes.toBytes("testbalances");
+  private static final byte[] FAMILY = Bytes.toBytes("f");
+  private static final byte[] COL = Bytes.toBytes("b");
+
+  private final int totalClients;
+  private final int iterations;
+
+  private Configuration conf;
+  private ZKClientService zkClient;
+  private TransactionServiceClient txClient;
+  private Connection conn;
+
+  public BalanceBooks(int totalClients, int iterations) {
+    this(totalClients, iterations, new ConfigurationFactory().get());
+  }
+
+  public BalanceBooks(int totalClients, int iterations, Configuration conf) {
+    this.totalClients = totalClients;
+    this.iterations = iterations;
+    this.conf = conf;
+  }
+
+  /**
+   * Sets up common resources required by all clients.
+   */
+  public void init() throws IOException {
+    Injector injector = Guice.createInjector(
+        new ConfigModule(conf),
+        new ZKModule(),
+        new DiscoveryModules().getDistributedModules(),
+        new TransactionModules().getDistributedModules(),
+        new TransactionClientModule()
+    );
+
+    zkClient = injector.getInstance(ZKClientService.class);
+    zkClient.startAndWait();
+    txClient = injector.getInstance(TransactionServiceClient.class);
+    conn = ConnectionFactory.createConnection(conf);
+    createTableIfNotExists(conf, TABLE, new byte[][]{ FAMILY });
+  }
+
+  /**
+   * Runs all clients and waits for them to complete.
+   */
+  public void run() throws IOException, InterruptedException {
+    List<Client> clients = new ArrayList<>(totalClients);
+    for (int i = 0; i < totalClients; i++) {
+      Client c = new Client(i, totalClients, iterations);
+      c.init(txClient, conn.getTable(TableName.valueOf(TABLE)));
+      c.start();
+      clients.add(c);
+    }
+
+    for (Client c : clients) {
+      c.join();
+      Closeables.closeQuietly(c);
+    }
+  }
+
+  /**
+   * Validates the current state of the data stored at the end of the test.  Each update by a client consists of two
+   * parts: a withdrawal of a random amount from a randomly select other account, and a corresponding to deposit to
+   * the client's own account.  So, if all the updates were performed consistently (no partial updates or partial
+   * rollbacks), then the total sum of all balances at the end should be 0.
+   */
+  public boolean verify() {
+    boolean success = false;
+    try {
+      TransactionAwareHTable table = new TransactionAwareHTable(conn.getTable(TableName.valueOf(TABLE)));
+      TransactionContext context = new TransactionContext(txClient, table);
+
+      LOG.info("VERIFYING BALANCES");
+      context.start();
+      long totalBalance = 0;
+
+      try (ResultScanner scanner = table.getScanner(new Scan())) {
+        for (Result r : scanner) {
+          if (!r.isEmpty()) {
+            int rowId = Bytes.toInt(r.getRow());
+            long balance = Bytes.toLong(r.getValue(FAMILY, COL));
+            totalBalance += balance;
+            LOG.info("Client #{}: balance = ${}", rowId, balance);
+          }
+        }
+      }
+      if (totalBalance == 0) {
+        LOG.info("PASSED!");
+        success = true;
+      } else {
+        LOG.info("FAILED! Total balance should be 0 but was {}", totalBalance);
+      }
+      context.finish();
+    } catch (Exception e) {
+      LOG.error("Failed verification check", e);
+    }
+    return success;
+  }
+
+  /**
+   * Frees up the underlying resources common to all clients.
+   */
+  public void close() {
+    try {
+      if (conn != null) {
+        conn.close();
+      }
+    } catch (IOException ignored) { }
+
+    if (zkClient != null) {
+      zkClient.stopAndWait();
+    }
+  }
+
+  protected void createTableIfNotExists(Configuration conf, byte[] tableName, byte[][] columnFamilies)
+      throws IOException {
+    try (Admin admin = this.conn.getAdmin()) {
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+      for (byte[] family : columnFamilies) {
+        HColumnDescriptor columnDesc = new HColumnDescriptor(family);
+        columnDesc.setMaxVersions(Integer.MAX_VALUE);
+        desc.addFamily(columnDesc);
+      }
+      desc.addCoprocessor(TransactionProcessor.class.getName());
+      admin.createTable(desc);
+    }
+  }
+
+  public static void main(String[] args) {
+    if (args.length != 2) {
+      System.err.println("Usage: java " + BalanceBooks.class.getName() + " <num clients> <iterations>");
+      System.err.println("\twhere <num clients> >= 2");
+      System.exit(1);
+    }
+
+    try (BalanceBooks bb = new BalanceBooks(Integer.parseInt(args[0]), Integer.parseInt(args[1]))) {
+      bb.init();
+      bb.run();
+      bb.verify();
+    } catch (Exception e) {
+      LOG.error("Failed during BalanceBooks run", e);
+    }
+  }
+
+  /**
+   * Represents a single client actor in the test.  Each client runs as a separate thread.
+   *
+   * For the given number of iterations, the client will:
+   * <ol>
+   *   <li>select a random other client from which to withdraw</li>
+   *   <li>select a random amount from 0 to MAX_AMOUNT</li>
+   *   <li>start a new transaction and: deduct the amount from the other client's acccount, and deposit
+   *       the same amount to its own account.</li>
+   * </ol>
+   *
+   * Since multiple clients operate concurrently and contend over a set of constrained resources
+   * (the client accounts), it is expected that a portion of the attempted transactions will encounter
+   * conflicts, due to a simultaneous deduction from or deposit to one the same accounts which has successfully
+   * committed first.  In this case, the updates from the transaction encountering the conflict should be completely
+   * rolled back, leaving the data in a consistent state.
+   */
+  private static class Client extends Thread implements Closeable {
+    private final int id;
+    private final int totalClients;
+    private final int iterations;
+
+    private final Random random = new Random();
+
+    private TransactionContext txContext;
+    private TransactionAwareHTable txTable;
+
+
+    public Client(int id, int totalClients, int iterations) {
+      this.id = id;
+      this.totalClients = totalClients;
+      this.iterations = iterations;
+    }
+
+    /**
+     * Sets up any resources needed by the individual client.
+     *
+     * @param txClient the transaction client to use in accessing the transaciton service
+     * @param table the HBase table instance to use for accessing storage
+     */
+    public void init(TransactionSystemClient txClient, Table table) {
+      txTable = new TransactionAwareHTable(table);
+      txContext = new TransactionContext(txClient, txTable);
+    }
+
+    public void run() {
+      try {
+        for (int i = 0; i < iterations; i++) {
+          runOnce();
+        }
+      } catch (TransactionFailureException e) {
+        LOG.error("Client #{}: Failed on exception", id, e);
+      }
+    }
+
+    /**
+     * Runs a single iteration of the client logic.
+     */
+    private void runOnce() throws TransactionFailureException {
+      int withdrawee = getNextWithdrawee();
+      int amount = getAmount();
+
+      try {
+        txContext.start();
+        long withdraweeBalance = getCurrentBalance(withdrawee);
+        long ownBalance = getCurrentBalance(id);
+        long withdraweeNew = withdraweeBalance - amount;
+        long ownNew = ownBalance + amount;
+
+        setBalance(withdrawee, withdraweeNew);
+        setBalance(id, ownNew);
+        LOG.debug("Client #{}: Withdrew ${} from #{}; withdrawee old={}, new={}; own old={}, new={}",
+            id, amount, withdrawee, withdraweeBalance, withdraweeNew, ownBalance, ownNew);
+        txContext.finish();
+
+      } catch (IOException ioe) {
+        LOG.error("Client #{}: Unhandled client failure", id, ioe);
+        txContext.abort();
+      } catch (TransactionConflictException tce) {
+        LOG.debug("CONFLICT: client #{} attempting to withdraw from #{}", id, withdrawee);
+        txContext.abort(tce);
+      } catch (TransactionFailureException tfe) {
+        LOG.error("Client #{}: Unhandled transaction failure", id, tfe);
+        txContext.abort(tfe);
+      }
+    }
+
+    private long getCurrentBalance(int id) throws IOException {
+      Result r = txTable.get(new Get(Bytes.toBytes(id)));
+      byte[] balanceBytes = r.getValue(FAMILY, COL);
+      if (balanceBytes == null) {
+        return 0;
+      }
+      return Bytes.toLong(balanceBytes);
+    }
+
+    private void setBalance(int id, long balance) throws IOException {
+      txTable.put(new Put(Bytes.toBytes(id)).addColumn(FAMILY, COL, Bytes.toBytes(balance)));
+    }
+
+    private int getNextWithdrawee() {
+      int next;
+      do {
+        next = random.nextInt(totalClients);
+      } while (next == id);
+      return next;
+    }
+
+    private int getAmount() {
+      return random.nextInt(MAX_AMOUNT);
+    }
+
+    public void close() throws IOException {
+      txTable.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java
new file mode 100644
index 0000000..a0e67d5
--- /dev/null
+++ b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains example applications for Tephra designed to illustrate sample Tephra usage
+ * and provide out-of-the-box sample applications which can be run to test cluster functionality.
+ *
+ * <p>Currently the following applications are provided:
+ *
+ * <ul>
+ *   <li><strong>BalanceBooks</strong> - this application runs a specified number of concurrent clients in separate
+ *     threads, which perform transactions to make withdrawals from each other's accounts and deposits to their own
+ *     accounts.  At the end of the test, the total value of all account balances is verified to be equal to zero,
+ *     which confirms that transactional integrity was not violated.
+ *   </li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ *   Note that, for simplicity, the examples package is currently hardcoded to compile against a specific HBase
+ *   version (currently 2.0).  In the future, we should provide Maven profiles to allow compiling the examples
+ *   against each of the supported HBase versions.
+ * </p>
+ */
+package org.apache.tephra.examples;

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java
----------------------------------------------------------------------
diff --git a/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java b/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java
new file mode 100644
index 0000000..4dfe107
--- /dev/null
+++ b/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java
@@ -0,0 +1,137 @@
+/*
+ * 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.examples;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Scopes;
+import com.google.inject.util.Modules;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.tephra.TransactionSystemClient;
+import org.apache.tephra.TxConstants;
+import org.apache.tephra.distributed.TransactionService;
+import org.apache.tephra.persist.InMemoryTransactionStateStorage;
+import org.apache.tephra.persist.TransactionStateStorage;
+import org.apache.tephra.runtime.ConfigModule;
+import org.apache.tephra.runtime.DiscoveryModules;
+import org.apache.tephra.runtime.TransactionClientModule;
+import org.apache.tephra.runtime.TransactionModules;
+import org.apache.tephra.runtime.ZKModule;
+import org.apache.tephra.util.Tests;
+import org.apache.twill.zookeeper.ZKClientService;
+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 static org.junit.Assert.assertTrue;
+
+/**
+ * Tests the {@link BalanceBooks} program.
+ */
+public class BalanceBooksTest {
+  private static final Logger LOG = LoggerFactory.getLogger(BalanceBooksTest.class);
+  private static HBaseTestingUtility testUtil;
+  private static TransactionService txService;
+  private static ZKClientService zkClientService;
+
+  @ClassRule
+  public static TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    testUtil = new HBaseTestingUtility();
+    Configuration conf = testUtil.getConfiguration();
+    conf.setBoolean(TxConstants.Manager.CFG_DO_PERSIST, false);
+    conf.set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, tmpFolder.newFolder().getAbsolutePath());
+
+    // 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();
+
+    String zkClusterKey = testUtil.getClusterKey(); // hostname:clientPort:parentZnode
+    String zkQuorum = zkClusterKey.substring(0, zkClusterKey.lastIndexOf(':'));
+    LOG.info("Zookeeper Quorum is running at {}", zkQuorum);
+    conf.set(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM, zkQuorum);
+
+    Injector injector = Guice.createInjector(
+        new ConfigModule(conf),
+        new ZKModule(),
+        new DiscoveryModules().getDistributedModules(),
+        Modules.override(new TransactionModules().getDistributedModules())
+            .with(new AbstractModule() {
+              @Override
+              protected void configure() {
+                bind(TransactionStateStorage.class).to(InMemoryTransactionStateStorage.class).in(Scopes.SINGLETON);
+              }
+            }),
+        new TransactionClientModule()
+    );
+
+    zkClientService = injector.getInstance(ZKClientService.class);
+    zkClientService.startAndWait();
+
+    // start a tx server
+    txService = injector.getInstance(TransactionService.class);
+    try {
+      LOG.info("Starting transaction service");
+      txService.startAndWait();
+    } catch (Exception e) {
+      LOG.error("Failed to start service: ", e);
+      throw e;
+    }
+
+    Tests.waitForTxReady(injector.getInstance(TransactionSystemClient.class));
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (txService != null) {
+      txService.stopAndWait();
+    }
+    if (zkClientService != null) {
+      zkClientService.stopAndWait();
+    }
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testBalanceBooks() throws Exception {
+    try (BalanceBooks bb = new BalanceBooks(5, 100, testUtil.getConfiguration())) {
+      bb.init();
+      bb.run();
+      assertTrue(bb.verify());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-examples/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-examples/pom.xml b/tephra-examples/pom.xml
index 6f10d3b..1f84e1e 100644
--- a/tephra-examples/pom.xml
+++ b/tephra-examples/pom.xml
@@ -38,6 +38,7 @@
     <module>hbase-1.2</module>
     <module>hbase-1.3</module>
     <module>hbase-1.4</module>
+    <module>hbase-2.0</module>
     <module>cdh-5.7</module>
     <module>cdh-5.8</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-hbase-compat-1.4/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-1.4/pom.xml b/tephra-hbase-compat-1.4/pom.xml
index d1ee108..873585d 100644
--- a/tephra-hbase-compat-1.4/pom.xml
+++ b/tephra-hbase-compat-1.4/pom.xml
@@ -127,6 +127,30 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-metrics-api</artifactId>
+      <version>${hbase14.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-metrics</artifactId>
+      <version>${hbase14.version}</version>
+      <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>${hbase14.version}</version>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-hbase-compat-2.0/pom.xml
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-2.0/pom.xml b/tephra-hbase-compat-2.0/pom.xml
new file mode 100644
index 0000000..1d4ffa0
--- /dev/null
+++ b/tephra-hbase-compat-2.0/pom.xml
@@ -0,0 +1,175 @@
+<?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.14.0-incubating-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tephra-hbase-compat-2.0</artifactId>
+  <name>Apache Tephra HBase 2.0 Compatibility</name>
+
+  <properties>
+    <hadoop.version>2.7.4</hadoop.version>
+    <hbase20.version>2.0.0-beta-1</hbase20.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>${hbase20.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>${hbase20.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>${hbase20.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase20.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>${hbase20.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-zookeeper</artifactId>
+      <version>${hbase20.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>${hbase20.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>asm</groupId>
+          <artifactId>asm</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-http</artifactId>
+      <version>${hbase20.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+      <plugins>
+        <!-- Compiler -->
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-compiler-plugin</artifactId>
+          <version>3.1</version>
+          <configuration>
+            <source>1.8</source>
+            <target>1.8</target>
+          </configuration>
+        </plugin>
+      </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java
new file mode 100644
index 0000000..96e4be3
--- /dev/null
+++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.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 2.0 version of {@link ConfigurationProvider}.
+ */
+public class HBase20ConfigurationProvider 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/db6ef6d2/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
new file mode 100644
index 0000000..d516334
--- /dev/null
+++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java
@@ -0,0 +1,185 @@
+/*
+ * 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 com.google.common.io.Closeables;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+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 Connection connection;
+  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, Table table,
+                             byte[] secondaryIndex) throws IOException {
+    secondaryIndexTableName = TableName.valueOf(table.getName().getNameAsString() + ".idx");
+    this.connection = ConnectionFactory.createConnection(table.getConfiguration());
+    Table secondaryIndexHTable = null;
+    try (Admin hBaseAdmin = this.connection.getAdmin()) {
+      if (!hBaseAdmin.tableExists(secondaryIndexTableName)) {
+        hBaseAdmin.createTable(TableDescriptorBuilder.newBuilder(secondaryIndexTableName).build());
+      }
+      secondaryIndexHTable = this.connection.getTable(secondaryIndexTableName);
+    } catch (Exception e) {
+      Closeables.closeQuietly(connection);
+      Throwables.propagate(e);
+    }
+
+    this.secondaryIndex = secondaryIndex;
+    this.transactionAwareHTable = new TransactionAwareHTable(table);
+    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(CellUtil.cloneValue(cell)));
+        }
+      }
+      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<Cell>>> familyMap = put.getFamilyCellMap().entrySet();
+        for (Map.Entry<byte [], List<Cell>> family : familyMap) {
+          for (Cell value : family.getValue()) {
+            if (Bytes.equals(value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(),
+                             secondaryIndex, 0, secondaryIndex.length)) {
+              byte[] secondaryRow = Bytes.add(CellUtil.cloneQualifier(value), DELIMITER,
+                                                    Bytes.add(CellUtil.cloneValue(value), DELIMITER,
+                                                          CellUtil.cloneRow(value)));
+              Put indexPut = new Put(secondaryRow);
+              indexPut.addColumn(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 {
+    Closeables.closeQuietly(connection);
+    try {
+      transactionAwareHTable.close();
+    } catch (IOException e) {
+      try {
+        secondaryIndexTable.close();
+      } catch (IOException ex) {
+        e.addSuppressed(ex);
+      }
+      throw e;
+    }
+    secondaryIndexTable.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
new file mode 100644
index 0000000..1f59ea0
--- /dev/null
+++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java
@@ -0,0 +1,721 @@
+/*
+ * 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.CompareOperator;
+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.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.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+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;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A Transaction Aware HTable implementation for HBase 2.0. 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 Table, TransactionAware {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionAwareHTable.class);
+  private final Table hTable;
+
+  /**
+   * Create a transactional aware instance of the passed HTable
+   * @param hTable underlying HBase table to use
+   */
+  public TransactionAwareHTable(Table 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(Table 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(Table 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(Table hTable, TxConstants.ConflictDetection conflictLevel,
+      boolean allowNonTransactional) {
+    super(conflictLevel, allowNonTransactional);
+    this.hTable = hTable;
+  }
+
+  /* AbstractTransactionAwareTable implementation */
+
+  @Override
+  protected byte[] getTableKey() {
+    return hTable.getName().getName();
+  }
+
+  @Override
+  protected boolean doCommit() throws IOException {
+    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.addFamilyVersion(change.getFamily(), transactionTimestamp);
+            break;
+          case COLUMN:
+            if (family != null && qualifier == null) {
+              rollbackDelete.addFamilyVersion(family, transactionTimestamp);
+            } else if (family != null && qualifier != null) {
+              rollbackDelete.addColumn(family, qualifier, transactionTimestamp);
+            }
+            break;
+          default:
+            throw new IllegalStateException("Unknown conflict detection level: " + conflictLevel);
+          }
+          rollbackDeletes.add(rollbackDelete);
+        }
+      }
+      hTable.delete(rollbackDeletes);
+      return true;
+    } finally {
+      tx = null;
+      changeSets.clear();
+    }
+  }
+
+  /* HTableInterface implementation */
+
+  @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 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 <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 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 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 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 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 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 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 boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+      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 boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+      byte[] value, Delete delete) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndDelete(row, family, qualifier, op, value, delete);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  @Override
+  public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+      byte[] value, RowMutations mutation) throws IOException {
+    if (allowNonTransactional) {
+      return hTable.checkAndMutate(row, family, qualifier, op, value, mutation);
+    } 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 void mutateRow(RowMutations rm) throws IOException {
+    if (tx == null) {
+      throw new IOException("Transaction not started");
+    }
+    RowMutations transactionalMutations = new RowMutations(rm.getRow());
+    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 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);
+  }
+
+  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]);
+  }
+
+  @Override
+  public TableDescriptor getDescriptor() throws IOException {
+    return hTable.getDescriptor();
+  }
+
+  @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 long getRpcTimeout(TimeUnit unit) {
+    return hTable.getRpcTimeout(unit);
+  }
+
+  @Override
+  public int getRpcTimeout() {
+    return hTable.getRpcTimeout();
+  }
+
+  @Override
+  public void setRpcTimeout(int rpcTimeout) {
+    hTable.setRpcTimeout(rpcTimeout);
+
+  }
+
+  @Override
+  public long getReadRpcTimeout(TimeUnit unit) {
+    return hTable.getReadRpcTimeout(unit);
+  }
+
+  @Override
+  public int getReadRpcTimeout() {
+    return hTable.getReadRpcTimeout();
+  }
+
+  @Override
+  public void setReadRpcTimeout(int readRpcTimeout) {
+    hTable.setReadRpcTimeout(readRpcTimeout);
+
+  }
+
+  @Override
+  public long getWriteRpcTimeout(TimeUnit unit) {
+    return hTable.getWriteRpcTimeout(unit);
+  }
+
+  @Override
+  public int getWriteRpcTimeout() {
+    return hTable.getWriteRpcTimeout();
+  }
+
+  @Override
+  public void setWriteRpcTimeout(int writeRpcTimeout) {
+    hTable.setWriteRpcTimeout(writeRpcTimeout);
+
+  }
+
+  @Override
+  public long getOperationTimeout(TimeUnit unit) {
+    return hTable.getOperationTimeout(unit);
+  }
+
+  @Override
+  public int getOperationTimeout() {
+    return hTable.getOperationTimeout();
+  }
+
+  @Override
+  public void setOperationTimeout(int operationTimeout) {
+    hTable.setOperationTimeout(operationTimeout);
+    ;
+  }
+
+  @Override
+  public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
+    if (allowNonTransactional) {
+      return hTable.checkAndMutate(row, family);
+    } else {
+      throw new UnsupportedOperationException("Operation is not supported transactionally");
+    }
+  }
+
+  // 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.addColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value),
+              tx.getWritePointer(), CellUtil.cloneValue(value));
+            addToChangeSet(txPut.getRow(), CellUtil.cloneFamily(value),
+              CellUtil.cloneQualifier(value));
+          }
+        }
+      }
+    }
+    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.addColumns(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.addFamily(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.addColumns(family, column.getKey(), transactionTimestamp);
+              addToChangeSet(deleteRow, family, column.getKey());
+            }
+          }
+        } else {
+          for (Cell value : entries) {
+            txDelete.addColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value),
+              transactionTimestamp);
+            addToChangeSet(deleteRow, CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value));
+          }
+        }
+      }
+    }
+    for (Map.Entry<String, byte[]> entry : delete.getAttributesMap().entrySet()) {
+      txDelete.setAttribute(entry.getKey(), entry.getValue());
+    }
+    txDelete.setDurability(delete.getDurability());
+    addToOperation(txDelete, tx);
+    return txDelete;
+  }
+
+  private List<? extends Row> transactionalizeActions(List<? extends Row> actions)
+      throws IOException {
+    List<Row> transactionalizedActions = new ArrayList<>(actions.size());
+    for (Row action : actions) {
+      if (action instanceof Get) {
+        transactionalizedActions.add(transactionalizeAction((Get) action));
+      } else if (action instanceof Put) {
+        transactionalizedActions.add(transactionalizeAction((Put) action));
+      } else if (action instanceof Delete) {
+        transactionalizedActions.add(transactionalizeAction((Delete) action));
+      } else {
+        transactionalizedActions.add(action);
+      }
+    }
+    return transactionalizedActions;
+  }
+
+}
+    

http://git-wip-us.apache.org/repos/asf/incubator-tephra/blob/db6ef6d2/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
----------------------------------------------------------------------
diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
new file mode 100644
index 0000000..4dbe924
--- /dev/null
+++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java
@@ -0,0 +1,129 @@
+/*
+ * 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)
+      && CellUtil.matchingFamily(cell, skipColumn)
+      && CellUtil.matchingQualifier(cell, skipColumn);
+  }
+
+  @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();
+  }
+
+  @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();
+  }
+}