You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2018/04/13 05:31:15 UTC

[1/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 5d4cb8041 -> 15fa00fa5


http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index ce70dd9..dee02d1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -25,42 +25,57 @@ import java.util.Map;
 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.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.transaction.PhoenixTransactionContext;
-import org.apache.phoenix.transaction.PhoenixTransactionalTable;
-import org.apache.phoenix.transaction.TephraTransactionTable;
 import org.apache.phoenix.transaction.TransactionFactory;
-import org.apache.tephra.util.TxUtils;
 
 public class TransactionUtil {
+    // All transaction providers must use an empty byte array as the family delete marker
+    // (see TxConstants.FAMILY_DELETE_QUALIFIER)
+    public static final byte[] FAMILY_DELETE_MARKER = HConstants.EMPTY_BYTE_ARRAY;
+    // All transaction providers must multiply timestamps by this constant.
+    // (see TxConstants.MAX_TX_PER_MS)
+    public static final int MAX_TRANSACTIONS_PER_MILLISECOND = 1000000;
+    // Constant used to empirically determine if a timestamp is a transactional or
+    // non transactional timestamp (see TxUtils.MAX_NON_TX_TIMESTAMP)
+    private static final long MAX_NON_TX_TIMESTAMP = (long) (System.currentTimeMillis() * 1.1);
+    
     private TransactionUtil() {
+        
     }
     
     public static boolean isTransactionalTimestamp(long ts) {
-        return !TxUtils.isPreExistingVersion(ts);
+        return ts >= MAX_NON_TX_TIMESTAMP;
     }
     
     public static boolean isDelete(Cell cell) {
-        return (CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY));
+        return CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY);
     }
     
-    public static long convertToNanoseconds(long serverTimeStamp) {
-        return serverTimeStamp * TransactionFactory.getTransactionProvider().getTransactionContext().getMaxTransactionsPerSecond();
+    public static boolean isDeleteFamily(Cell cell) {
+        return CellUtil.matchingQualifier(cell, FAMILY_DELETE_MARKER) && CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY);
     }
     
-    public static long convertToMilliseconds(long serverTimeStamp) {
-        return serverTimeStamp / TransactionFactory.getTransactionProvider().getTransactionContext().getMaxTransactionsPerSecond();
+    private static Cell newDeleteFamilyMarker(byte[] row, byte[] family, long timestamp) {
+        return CellUtil.createCell(row, family, FAMILY_DELETE_MARKER, timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
     }
     
-    public static PhoenixTransactionalTable getPhoenixTransactionTable(PhoenixTransactionContext phoenixTransactionContext, Table htable, PTable pTable) {
-        return new TephraTransactionTable(phoenixTransactionContext, htable, pTable);
+    private static Cell newDeleteColumnMarker(byte[] row, byte[] family, byte[] qualifier, long timestamp) {
+        return CellUtil.createCell(row, family, qualifier, timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
+    }
+
+    public static long convertToNanoseconds(long serverTimeStamp) {
+        return serverTimeStamp * MAX_TRANSACTIONS_PER_MILLISECOND;
+    }
+    
+    public static long convertToMilliseconds(long serverTimeStamp) {
+        return serverTimeStamp / MAX_TRANSACTIONS_PER_MILLISECOND;
     }
     
     // we resolve transactional tables at the txn read pointer
@@ -83,14 +98,14 @@ public class TransactionUtil {
 		return  txInProgress ? convertToMilliseconds(mutationState.getInitialWritePointer()) : result.getMutationTime();
 	}
 
-	public static Long getTableTimestamp(PhoenixConnection connection, boolean transactional) throws SQLException {
+	public static Long getTableTimestamp(PhoenixConnection connection, boolean transactional, TransactionFactory.Provider provider) throws SQLException {
 		Long timestamp = null;
 		if (!transactional) {
 			return timestamp;
 		}
 		MutationState mutationState = connection.getMutationState();
 		if (!mutationState.isTransactionStarted()) {
-			mutationState.startTransaction();
+			mutationState.startTransaction(provider);
 		}
 		timestamp = convertToMilliseconds(mutationState.getInitialWritePointer());
 		return timestamp;
@@ -108,7 +123,7 @@ public class TransactionUtil {
                         if (deleteMarker == null) {
                             deleteMarker = new Put(mutation.getRow());
                         }
-                        deleteMarker.add(TransactionFactory.getTransactionProvider().newDeleteFamilyMarker(
+                        deleteMarker.add(newDeleteFamilyMarker(
                                 deleteMarker.getRow(), 
                                 family, 
                                 familyCells.get(0).getTimestamp()));
@@ -119,7 +134,7 @@ public class TransactionUtil {
                             if (deleteMarker == null) {
                                 deleteMarker = new Put(mutation.getRow());
                             }
-                            deleteMarker.add(TransactionFactory.getTransactionProvider().newDeleteColumnMarker(
+                            deleteMarker.add(newDeleteColumnMarker(
                                     deleteMarker.getRow(),
                                     family,
                                     CellUtil.cloneQualifier(cell), 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
index 64045ae..d88a915 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
@@ -63,8 +63,8 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.EncodedCQCounter;
-import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
@@ -261,7 +261,7 @@ public class CorrelatePlanTest {
                     PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                     null, null, columns, null, null, Collections.<PTable>emptyList(),
                     false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                    null, null, true, false, 0, 0L, Boolean.FALSE, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
+                    null, null, true, null, 0, 0L, Boolean.FALSE, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
             TableRef sourceTable = new TableRef(pTable);
             List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
index 1a7132c..017e6c8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
@@ -50,7 +50,6 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.SelectStatement;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnImpl;
@@ -58,11 +57,11 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.EncodedCQCounter;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
-import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.junit.Test;
@@ -183,7 +182,7 @@ public class LiteralResultIteratorPlanTest {
             PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
                     MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
                     Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
-                    false, null, null, null, true, false, 0, 0L, false, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
+                    false, null, null, null, true, null, 0, 0L, false, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
             TableRef sourceTable = new TableRef(pTable);
             List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index f3674c6..7e0408e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -130,7 +130,6 @@ import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
-import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -168,7 +167,6 @@ public abstract class BaseTest {
     
     private static final Map<String,String> tableDDLMap;
     private static final Logger logger = LoggerFactory.getLogger(BaseTest.class);
-    protected static final int DEFAULT_TXN_TIMEOUT_SECONDS = 30;
     @ClassRule
     public static TemporaryFolder tmpFolder = new TemporaryFolder();
     private static final int dropTableTimeout = 300; // 5 mins should be long enough.
@@ -414,18 +412,6 @@ public abstract class BaseTest {
         return url;
     }
     
-    private static void tearDownTxManager() throws SQLException {
-        TransactionFactory.getTransactionProvider().getTransactionContext().tearDownTxManager();
-    }
-
-    protected static void setTxnConfigs() throws IOException {
-        TransactionFactory.getTransactionProvider().getTransactionContext().setTxnConfigs(config, tmpFolder.newFolder().getAbsolutePath(), DEFAULT_TXN_TIMEOUT_SECONDS);
-    }
-
-    protected static void setupTxManager() throws SQLException, IOException {
-        TransactionFactory.getTransactionProvider().getTransactionContext().setupTxManager(config, getUrl());
-    }
-
     private static String checkClusterInitialized(ReadOnlyProps serverProps) throws Exception {
         if (!clusterInitialized) {
             url = setUpTestCluster(config, serverProps);
@@ -434,10 +420,6 @@ public abstract class BaseTest {
         return url;
     }
 
-    private static void checkTxManagerInitialized(ReadOnlyProps clientProps) throws SQLException, IOException {
-        setupTxManager();
-    }
-
     /**
      * Set up the test hbase cluster.
      * @return url to be used by clients to connect to the cluster.
@@ -476,11 +458,6 @@ public abstract class BaseTest {
         final HBaseTestingUtility u = utility;
         try {
             destroyDriver();
-            try {
-                tearDownTxManager();
-            } catch (Throwable t) {
-                logger.error("Exception caught when shutting down tx manager", t);
-            }
             utility = null;
             clusterInitialized = false;
         } finally {
@@ -519,9 +496,7 @@ public abstract class BaseTest {
     
     protected static void setUpTestDriver(ReadOnlyProps serverProps, ReadOnlyProps clientProps) throws Exception {
         if (driver == null) {
-            setTxnConfigs();
             String url = checkClusterInitialized(serverProps);
-            checkTxManagerInitialized(serverProps);
             driver = initAndRegisterTestDriver(url, clientProps);
         }
     }
@@ -593,6 +568,7 @@ public abstract class BaseTest {
         conf.set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS, DEFAULT_RPC_SCHEDULER_FACTORY);
         conf.setLong(HConstants.ZK_SESSION_TIMEOUT, 10 * HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
         conf.setLong(HConstants.ZOOKEEPER_TICK_TIME, 6 * 1000);
+        
         // override any defaults based on overrideProps
         for (Entry<String,String> entry : overrideProps) {
             conf.set(entry.getKey(), entry.getValue());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index c93e56e..a7569f7 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -20,9 +20,12 @@ package org.apache.phoenix.query;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_SPOOL_DIRECTORY;
 import static org.apache.phoenix.query.QueryServicesOptions.withDefaults;
 
+import org.apache.curator.shaded.com.google.common.io.Files;
 import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.tephra.TxConstants;
+import org.apache.twill.internal.utils.Networks;
 
 
 /**
@@ -69,6 +72,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
      * because we want to control it's execution ourselves
      */
     public static final long DEFAULT_INDEX_REBUILD_TASK_INITIAL_DELAY = Long.MAX_VALUE;
+    public static final int DEFAULT_TXN_TIMEOUT_SECONDS = 30;
 
     
     /**
@@ -117,7 +121,16 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .setHConnectionPoolMaxSize(DEFAULT_HCONNECTION_POOL_MAX_SIZE)
                 .setMaxThreadsPerHTable(DEFAULT_HTABLE_MAX_THREADS)
                 .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME)
-                .setIndexRebuildTaskInitialDelay(DEFAULT_INDEX_REBUILD_TASK_INITIAL_DELAY);
+                .setIndexRebuildTaskInitialDelay(DEFAULT_INDEX_REBUILD_TASK_INITIAL_DELAY)
+                // setup default configs for Tephra
+                .set(TxConstants.Manager.CFG_DO_PERSIST, false)
+                .set(TxConstants.Service.CFG_DATA_TX_CLIENT_RETRY_STRATEGY, "n-times")
+                .set(TxConstants.Service.CFG_DATA_TX_CLIENT_ATTEMPTS, 1)
+                .set(TxConstants.Service.CFG_DATA_TX_BIND_PORT, Networks.getRandomPort())
+                .set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, Files.createTempDir().getAbsolutePath())
+                .set(TxConstants.Manager.CFG_TX_TIMEOUT, DEFAULT_TXN_TIMEOUT_SECONDS)
+                .set(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5L)
+                ;
     }
     
     public QueryServicesTestImpl(ReadOnlyProps defaultProps, ReadOnlyProps overrideProps) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 29d964b..4a56637 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -782,7 +782,7 @@ public class TestUtil {
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
         MutationState mutationState = pconn.getMutationState();
         if (table.isTransactional()) {
-            mutationState.startTransaction();
+            mutationState.startTransaction(table.getTransactionProvider());
         }
         try (Table htable = mutationState.getHTable(table)) {
             byte[] markerRowKey = Bytes.toBytes("TO_DELETE");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index ba9e0b4..16381dd 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -100,6 +100,7 @@ message PTable {
   optional bytes encodingScheme = 35;
   repeated EncodedCQCounter encodedCQCounters = 36;
   optional bool useStatsForParallelization = 37;
+  optional int32 transactionProvider = 38;
 }
 
 message EncodedCQCounter {


[3/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 1c3bff6..e062729 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -82,6 +82,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION;
@@ -224,6 +225,9 @@ import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
+import org.apache.phoenix.transaction.PhoenixTransactionProvider;
+import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.CursorUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
@@ -282,7 +286,7 @@ public class MetaDataClient {
                     INDEX_TYPE + "," +
                     STORE_NULLS + "," +
                     BASE_COLUMN_COUNT + "," +
-                    TRANSACTIONAL + "," +
+                    TRANSACTION_PROVIDER + "," +
                     UPDATE_CACHE_FREQUENCY + "," +
                     IS_NAMESPACE_MAPPED + "," +
                     AUTO_PARTITION_SEQ +  "," +
@@ -574,14 +578,11 @@ public class MetaDataClient {
         } catch (TableNotFoundException e) {
         }
 
-        boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
         // TODO if system tables become transactional remove the check
-        boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
-        if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
-            connection.getMutationState().startTransaction();
+        boolean isTransactional = (table!=null && table.isTransactional());
+        if (isTransactional) {
+            connection.getMutationState().startTransaction(table.getTransactionProvider());
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
         // Do not make rpc to getTable if
@@ -634,16 +635,20 @@ public class MetaDataClient {
                 result =
                         queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp,
                             resolvedTimestamp);
-                // if the table was assumed to be transactional, but is actually not transactional
-                // then re-resolve as of the right timestamp (and vice versa)
-                if (table == null && result.getTable() != null
-                        && result.getTable().isTransactional() != isTransactional) {
-                    result =
-                            queryServices.getTable(tenantId, schemaBytes, tableBytes,
-                                tableTimestamp,
-                                TransactionUtil.getResolvedTimestamp(connection,
-                                    result.getTable().isTransactional(),
-                                    HConstants.LATEST_TIMESTAMP));
+                // if the table was assumed to be non transactional, but is actually transactional
+                // then re-resolve as of the right timestamp
+                if (result.getTable() != null
+                        && result.getTable().isTransactional()
+                        && !isTransactional) {
+                    long resolveTimestamp = TransactionUtil.getResolvedTimestamp(connection,
+                            result.getTable().isTransactional(),
+                            HConstants.LATEST_TIMESTAMP);
+                    // Reresolve if table timestamp is past timestamp as of which we should see data
+                    if (result.getTable().getTimeStamp() >= resolveTimestamp) {
+                        result =
+                                queryServices.getTable(tenantId, schemaBytes, tableBytes,
+                                    tableTimestamp, resolveTimestamp);
+                    }
                 }
 
                 if (SYSTEM_CATALOG_SCHEMA.equals(schemaName)) {
@@ -1238,8 +1243,8 @@ public class MetaDataClient {
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
-                public boolean isTransactional() {
-                    return false;
+                public TransactionFactory.Provider getTransactionProvider() {
+                    return null;
                 }
             };
             TableRef tableRef = new TableRef(null, nonTxnLogicalTable, clientTimeStamp, false);
@@ -1863,7 +1868,7 @@ public class MetaDataClient {
             long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
             boolean multiTenant = false;
             boolean storeNulls = false;
-            boolean transactional = (parent!= null) ? parent.isTransactional() : false;
+            TransactionFactory.Provider transactionProvider = (parent!= null) ? parent.getTransactionProvider() : null;
             Integer saltBucketNum = null;
             String defaultFamilyName = null;
             boolean isImmutableRows = false;
@@ -1879,7 +1884,7 @@ public class MetaDataClient {
             QualifierEncodingScheme encodingScheme = NON_ENCODED_QUALIFIERS;
             ImmutableStorageScheme immutableStorageScheme = ONE_CELL_PER_COLUMN;
             if (parent != null && tableType == PTableType.INDEX) {
-                timestamp = TransactionUtil.getTableTimestamp(connection, transactional);
+                timestamp = TransactionUtil.getTableTimestamp(connection, transactionProvider != null, transactionProvider);
                 storeNulls = parent.getStoreNulls();
                 isImmutableRows = parent.isImmutableRows();
                 isAppendOnlySchema = parent.isAppendOnlySchema();
@@ -2020,31 +2025,45 @@ public class MetaDataClient {
                 storeNulls = storeNullsProp;
             }
             Boolean transactionalProp = (Boolean) TableProperty.TRANSACTIONAL.getValue(tableProps);
-            if (transactionalProp != null && parent != null) {
+            TransactionFactory.Provider transactionProviderProp = (TransactionFactory.Provider) TableProperty.TRANSACTION_PROVIDER.getValue(tableProps);
+            if ((transactionalProp != null || transactionProviderProp != null) && parent != null) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.ONLY_TABLE_MAY_BE_DECLARED_TRANSACTIONAL)
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
             if (parent == null) {
-                if (transactionalProp == null) {
+                boolean transactional;
+                if (transactionProviderProp != null) {
+                    transactional = true;
+                } else if (transactionalProp == null) {
                     transactional = connection.getQueryServices().getProps().getBoolean(
                                     QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
                                     QueryServicesOptions.DEFAULT_TABLE_ISTRANSACTIONAL);
                 } else {
                     transactional = transactionalProp;
                 }
+                if (transactional) {
+                    if (transactionProviderProp == null) {
+                        transactionProvider = (TransactionFactory.Provider)TableProperty.TRANSACTION_PROVIDER.getValue(
+                                connection.getQueryServices().getProps().get(
+                                        QueryServices.DEFAULT_TRANSACTION_PROVIDER_ATTRIB,
+                                        QueryServicesOptions.DEFAULT_TRANSACTION_PROVIDER));
+                    } else {
+                        transactionProvider = transactionProviderProp;
+                    }
+                }
             }
             boolean transactionsEnabled = connection.getQueryServices().getProps().getBoolean(
                                             QueryServices.TRANSACTIONS_ENABLED,
                                             QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
             // can't create a transactional table if transactions are not enabled
-            if (!transactionsEnabled && transactional) {
+            if (!transactionsEnabled && transactionProvider != null) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED)
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
             // can't create a transactional table if it has a row timestamp column
-            if (pkConstraint.getNumColumnsWithRowTimestamp() > 0 && transactional) {
+            if (pkConstraint.getNumColumnsWithRowTimestamp() > 0 && transactionProvider != null) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP)
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
@@ -2052,8 +2071,9 @@ public class MetaDataClient {
 
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
-            tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
-            if (transactional) {
+            tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactionProvider != null);
+            if (transactionProvider != null) {
+                // TODO: for Omid
                 // If TTL set, use Tephra TTL property name instead
                 Object ttl = commonFamilyProps.remove(ColumnFamilyDescriptorBuilder.TTL);
                 if (ttl != null) {
@@ -2065,7 +2085,7 @@ public class MetaDataClient {
                     (Boolean) TableProperty.USE_STATS_FOR_PARALLELIZATION.getValue(tableProps);
 
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || allocateIndexId;
-            if (transactional) {
+            if (transactionProvider != null) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -2100,7 +2120,7 @@ public class MetaDataClient {
                     }
                 }
             }
-            timestamp = timestamp==null ? TransactionUtil.getTableTimestamp(connection, transactional) : timestamp;
+            timestamp = timestamp==null ? TransactionUtil.getTableTimestamp(connection, transactionProvider != null, transactionProvider) : timestamp;
 
             // Delay this check as it is supported to have IMMUTABLE_ROWS and SALT_BUCKETS defined on views
             if (sharedTable) {
@@ -2483,7 +2503,7 @@ public class MetaDataClient {
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, null, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
             
@@ -2623,7 +2643,11 @@ public class MetaDataClient {
             } else {
                 tableUpsert.setInt(19, BASE_TABLE_BASE_COLUMN_COUNT);
             }
-            tableUpsert.setBoolean(20, transactional);
+            if (transactionProvider == null) {
+                tableUpsert.setNull(20, Types.TINYINT);
+            } else {
+                tableUpsert.setByte(20, transactionProvider.getCode());
+            }
             tableUpsert.setLong(21, updateCacheFrequency);
             tableUpsert.setBoolean(22, isNamespaceMapped);
             if (autoPartitionSeq == null) {
@@ -2748,7 +2772,7 @@ public class MetaDataClient {
                         PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         parent == null ? null : parent.getSchemaName(), parent == null ? null : parent.getTableName(), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelizationProp);
+                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelizationProp);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -3252,8 +3276,8 @@ public class MetaDataClient {
                 changingPhoenixTableProperty = evaluateStmtProperties(metaProperties,metaPropertiesEvaluated,table,schemaName,tableName);
                 // If changing isImmutableRows to true or it's not being changed and is already true
                 boolean willBeImmutableRows = Boolean.TRUE.equals(metaPropertiesEvaluated.getIsImmutableRows()) || (metaPropertiesEvaluated.getIsImmutableRows() == null && table.isImmutableRows());
-
-                Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || metaProperties.getNonTxToTx());
+                boolean willBeTxnl = metaProperties.getNonTxToTx();
+                Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || willBeTxnl, table.isTransactional() ? table.getTransactionProvider() : metaPropertiesEvaluated.getTransactionProvider());
                 int numPkColumnsAdded = 0;
                 List<PColumn> columns = Lists.newArrayListWithExpectedSize(numCols);
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
@@ -4116,6 +4140,8 @@ public class MetaDataClient {
                         metaProperties.setStoreNullsProp((Boolean)value);
                     } else if (propName.equals(TRANSACTIONAL)) {
                         metaProperties.setIsTransactionalProp((Boolean)value);
+                    } else if (propName.equals(TRANSACTION_PROVIDER)) {
+                        metaProperties.setTransactionProviderProp((TransactionFactory.Provider) value);
                     } else if (propName.equals(UPDATE_CACHE_FREQUENCY)) {
                         metaProperties.setUpdateCacheFrequencyProp((Long)value);
                     } else if (propName.equals(GUIDE_POSTS_WIDTH)) {
@@ -4238,6 +4264,22 @@ public class MetaDataClient {
                             .setSchemaName(schemaName).setTableName(tableName)
                             .build().buildException();
                 }
+                TransactionFactory.Provider provider = metaProperties.getTransactionProviderProp();
+                if (provider == null) {
+                    provider = (Provider)
+                            TableProperty.TRANSACTION_PROVIDER.getValue(
+                                    connection.getQueryServices().getProps().get(
+                                            QueryServices.DEFAULT_TRANSACTION_PROVIDER_ATTRIB,
+                                            QueryServicesOptions.DEFAULT_TRANSACTION_PROVIDER));
+                    metaPropertiesEvaluated.setTransactionProvider(provider);
+                }
+                if (provider.getTransactionProvider().isUnsupported(PhoenixTransactionProvider.Feature.ALTER_NONTX_TO_TX)) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TABLE_FROM_NON_TXN_TO_TXNL)
+                        .setMessage(provider.name() + ". ")
+                        .setSchemaName(schemaName)
+                        .setTableName(tableName)
+                        .build().buildException();
+                }
                 changingPhoenixTableProperty = true;
                 metaProperties.setNonTxToTx(true);
             }
@@ -4250,6 +4292,7 @@ public class MetaDataClient {
         private Boolean multiTenantProp = null;
         private Boolean disableWALProp = null;
         private Boolean storeNullsProp = null;
+        private TransactionFactory.Provider transactionProviderProp = null;
         private Boolean isTransactionalProp = null;
         private Long updateCacheFrequencyProp = null;
         private Boolean appendOnlySchemaProp = null;
@@ -4290,6 +4333,14 @@ public class MetaDataClient {
             this.storeNullsProp = storeNullsProp;
         }
 
+        public TransactionFactory.Provider getTransactionProviderProp() {
+            return transactionProviderProp;
+        }
+
+        public void setTransactionProviderProp(TransactionFactory.Provider transactionProviderProp) {
+            this.transactionProviderProp = transactionProviderProp;
+        }
+
         public Boolean getIsTransactionalProp() {
             return isTransactionalProp;
         }
@@ -4359,6 +4410,7 @@ public class MetaDataClient {
         private Boolean storeNulls = null;
         private Boolean useStatsForParallelization = null;
         private Boolean isTransactional = null;
+        private TransactionFactory.Provider transactionProvider = null;
 
         public Boolean getIsImmutableRows() {
             return isImmutableRows;
@@ -4439,6 +4491,15 @@ public class MetaDataClient {
         public void setIsTransactional(Boolean isTransactional) {
             this.isTransactional = isTransactional;
         }
+        
+        public TransactionFactory.Provider getTransactionProvider() {
+            return transactionProvider;
+        }
+
+        public void setTransactionProvider(TransactionFactory.Provider transactionProvider) {
+            this.transactionProvider = transactionProvider;
+        }
+
     }
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index 7e186ad..af78612 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -40,6 +40,7 @@ import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
 import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -680,6 +681,7 @@ public interface PTable extends PMetaDataEntity {
     boolean isMultiTenant();
     boolean getStoreNulls();
     boolean isTransactional();
+    TransactionFactory.Provider getTransactionProvider();
 
     ViewType getViewType();
     String getViewStatement();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 633595a..a451918 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -135,7 +135,7 @@ public class PTableImpl implements PTable {
     private boolean disableWAL;
     private boolean multiTenant;
     private boolean storeNulls;
-    private boolean isTransactional;
+    private TransactionFactory.Provider transactionProvider;
     private ViewType viewType;
     private Short viewIndexId;
     private int estimatedSize;
@@ -228,7 +228,7 @@ public class PTableImpl implements PTable {
         init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
             this.schemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
             null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-            isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+            transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
     }
 
     public PTableImpl(long timeStamp) { // For delete marker
@@ -271,7 +271,7 @@ public class PTableImpl implements PTable {
                     table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
                     indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                    table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
+                    table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), updateCacheFrequency,
                     table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
         }
 
@@ -281,7 +281,7 @@ public class PTableImpl implements PTable {
                 table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), parentSchemaName, table.getParentTableName(),
                 indexes, table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), viewStatement,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -291,7 +291,7 @@ public class PTableImpl implements PTable {
                 table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
     
@@ -301,7 +301,7 @@ public class PTableImpl implements PTable {
                 table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -311,7 +311,7 @@ public class PTableImpl implements PTable {
                 table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -321,7 +321,7 @@ public class PTableImpl implements PTable {
                 sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(),
                 table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
                 table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -331,18 +331,18 @@ public class PTableImpl implements PTable {
                 sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                table.getIndexType(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(),
+                table.getIndexType(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(),
                 table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
     
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled,
-            boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped) throws SQLException {
+            boolean isMultitenant, boolean storeNulls, TransactionFactory.Provider transactionProvider, long updateCacheFrequency, boolean isNamespaceMapped) throws SQLException {
         return new PTableImpl(
                 table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
                 sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 isWalDisabled, isMultitenant, storeNulls, table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), isTransactional, updateCacheFrequency, table.getIndexDisableTimestamp(), 
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), transactionProvider, updateCacheFrequency, table.getIndexDisableTimestamp(), 
                 isNamespaceMapped, table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
     
@@ -353,7 +353,7 @@ public class PTableImpl implements PTable {
                 table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -364,7 +364,7 @@ public class PTableImpl implements PTable {
                 table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), rowKeyOrderOptimizable, table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
+                table.getBaseColumnCount(), rowKeyOrderOptimizable, table.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
                 table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -375,7 +375,7 @@ public class PTableImpl implements PTable {
                 table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
                 table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
@@ -384,12 +384,12 @@ public class PTableImpl implements PTable {
             Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
-            IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
+            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
             long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataSchemaName,
                 dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
-                indexType, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, isTransactional,
+                indexType, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, transactionProvider,
                 updateCacheFrequency,indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
     }
 
@@ -398,7 +398,7 @@ public class PTableImpl implements PTable {
             Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
-            IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
+            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
             int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
             String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
             QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization)
@@ -406,7 +406,7 @@ public class PTableImpl implements PTable {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
                 bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
                 defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
-                indexType, baseColumnCount, rowKeyOrderOptimizable, isTransactional, updateCacheFrequency, 
+                indexType, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, 
                 indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
     }
 
@@ -415,13 +415,13 @@ public class PTableImpl implements PTable {
             PName parentSchemaName, PName parentTableName, List<PTable> indexes, boolean isImmutableRows,
             List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
-            int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
+            int baseColumnCount, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
             long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, 
             QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
                 parentSchemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-                isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, 
+                transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, 
                 qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
     }
     
@@ -455,7 +455,7 @@ public class PTableImpl implements PTable {
             PName pkName, Integer bucketNum, Collection<PColumn> columns, PName parentSchemaName, PName parentTableName,
             List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
-            IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency, long indexDisableTimestamp, 
+            IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency, long indexDisableTimestamp, 
             boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
             EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         Preconditions.checkNotNull(schemaName);
@@ -487,7 +487,7 @@ public class PTableImpl implements PTable {
         this.viewType = viewType;
         this.viewIndexId = viewIndexId;
         this.indexType = indexType;
-        this.isTransactional = isTransactional;
+        this.transactionProvider = transactionProvider;
         this.rowKeyOrderOptimizable = rowKeyOrderOptimizable;
         this.updateCacheFrequency = updateCacheFrequency;
         this.isNamespaceMapped = isNamespaceMapped;
@@ -1278,7 +1278,13 @@ public class PTableImpl implements PTable {
         boolean disableWAL = table.getDisableWAL();
         boolean multiTenant = table.getMultiTenant();
         boolean storeNulls = table.getStoreNulls();
-        boolean isTransactional = table.getTransactional();
+        TransactionFactory.Provider transactionProvider = null;
+        if (table.hasTransactionProvider()) {
+            transactionProvider = TransactionFactory.Provider.fromCode(table.getTransactionProvider());
+        } else if (table.hasTransactional()) {
+            // For backward compatibility prior to transactionProvider field
+            transactionProvider = TransactionFactory.Provider.TEPHRA;
+        }
         ViewType viewType = null;
         String viewStatement = null;
         List<PName> physicalNames = Collections.emptyList();
@@ -1351,7 +1357,7 @@ public class PTableImpl implements PTable {
                 (bucketNum == NO_SALTING) ? null : bucketNum, columns, parentSchemaName, parentTableName, indexes,
                         isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
                         multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-                        isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, 
+                        transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, 
                         isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedColumnQualifierCounter, useStatsForParallelization);
             return result;
         } catch (SQLException e) {
@@ -1417,7 +1423,9 @@ public class PTableImpl implements PTable {
       builder.setDisableWAL(table.isWALDisabled());
       builder.setMultiTenant(table.isMultiTenant());
       builder.setStoreNulls(table.getStoreNulls());
-      builder.setTransactional(table.isTransactional());
+      if (table.getTransactionProvider() != null) {
+          builder.setTransactionProvider(table.getTransactionProvider().getCode());
+      }
       if(table.getType() == PTableType.VIEW){
         builder.setViewType(ByteStringer.wrap(new byte[]{table.getViewType().getSerializedValue()}));
       }
@@ -1472,8 +1480,13 @@ public class PTableImpl implements PTable {
     }
 
     @Override
-    public boolean isTransactional() {
-        return isTransactional;
+    public TransactionFactory.Provider getTransactionProvider() {
+        return transactionProvider;
+    }
+    
+    @Override
+    public final boolean isTransactional() {
+        return transactionProvider != null;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
index c500b2e..78b9beb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
@@ -33,6 +33,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.SchemaUtil;
 
 public enum TableProperty {
@@ -94,6 +95,23 @@ public enum TableProperty {
         }
     },
     
+    TRANSACTION_PROVIDER(PhoenixDatabaseMetaData.TRANSACTION_PROVIDER, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) {
+        @Override
+        public Object getPTableValue(PTable table) {
+            return table.getTransactionProvider();
+        }
+        @Override
+        public Object getValue(Object value) {
+            try {
+                return value == null ? null : TransactionFactory.Provider.valueOf(value.toString());
+            } catch (IllegalArgumentException e) {
+                throw new RuntimeException(new SQLExceptionInfo.Builder(SQLExceptionCode.UNKNOWN_TRANSACTION_PROVIDER)
+                .setMessage(value.toString())
+                .build().buildException());
+            }
+        }
+    },
+
     UPDATE_CACHE_FREQUENCY(PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY, true, true, true) {
 	    @Override
         public Object getValue(Object value) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
index a8b913f..e74c97a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
@@ -17,16 +17,11 @@
  */
 package org.apache.phoenix.transaction;
 
-import java.io.IOException;
 import java.sql.SQLException;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.twill.zookeeper.ZKClientService;
-import org.slf4j.Logger;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 
 public class OmidTransactionContext implements PhoenixTransactionContext {
 
@@ -55,7 +50,7 @@ public class OmidTransactionContext implements PhoenixTransactionContext {
     }
 
     @Override
-    public void commitDDLFence(PTable dataTable, Logger logger) throws SQLException {
+    public void commitDDLFence(PTable dataTable) throws SQLException {
         // TODO Auto-generated method stub
 
     }
@@ -115,59 +110,24 @@ public class OmidTransactionContext implements PhoenixTransactionContext {
     }
 
     @Override
-    public long getMaxTransactionsPerSecond() {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public boolean isPreExistingVersion(long version) {
-        // TODO Auto-generated method stub
-        return false;
+    public Provider getProvider() {
+        return Provider.OMID;
     }
 
     @Override
-    public RegionObserver getCoprocessor() {
-        // TODO Auto-generated method stub
+    public PhoenixTransactionContext newTransactionContext(PhoenixTransactionContext contex, boolean subTask) {
         return null;
     }
 
     @Override
-    public void setInMemoryTransactionClient(Configuration config) {
+    public void markDMLFence(PTable dataTable) {
         // TODO Auto-generated method stub
         
     }
 
     @Override
-    public ZKClientService setTransactionClient(Configuration config, ReadOnlyProps props,
-            ConnectionInfo connectionInfo) {
-        // TODO Auto-generated method stub
-        
-        return null;
-        
-    }
-
-    @Override
-    public byte[] getFamilyDeleteMarker() {
+    public Table getTransactionalTable(Table htable, boolean isImmutable) {
         // TODO Auto-generated method stub
         return null;
     }
-
-    @Override
-    public void setTxnConfigs(Configuration config, String tmpFolder, int defaultTxnTimeoutSeconds) throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void setupTxManager(Configuration config, String url) throws SQLException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void tearDownTxManager() {
-        // TODO Auto-generated method stub
-
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
index fce3d9a..c211661 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
@@ -19,34 +19,28 @@ package org.apache.phoenix.transaction;
 
 import java.io.IOException;
 
-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.client.Table;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 
-public class OmidTransactionProvider implements TransactionProvider {
+public class OmidTransactionProvider implements PhoenixTransactionProvider {
     private static final OmidTransactionProvider INSTANCE = new OmidTransactionProvider();
-    
+
     public static final OmidTransactionProvider getInstance() {
         return INSTANCE;
     }
-    
+
     private OmidTransactionProvider() {
     }
-    
-    @Override
-    public PhoenixTransactionContext getTransactionContext()  {
-        return new OmidTransactionContext();
-    }
 
     @Override
     public PhoenixTransactionContext getTransactionContext(byte[] txnBytes) throws IOException {
         //return new OmidTransactionContext(txnBytes);
         return null;
     }
-    
+
     @Override
     public PhoenixTransactionContext getTransactionContext(PhoenixConnection connection) {
         //return new OmidTransactionContext(connection);
@@ -54,25 +48,37 @@ public class OmidTransactionProvider implements TransactionProvider {
     }
 
     @Override
-    public PhoenixTransactionContext getTransactionContext(PhoenixTransactionContext contex, PhoenixConnection connection, boolean subTask) {
-        //return new OmidTransactionContext(contex, connection, subTask);
+    public PhoenixTransactionClient getTransactionClient(Configuration config, ConnectionInfo connectionInfo) {
+        // TODO Auto-generated method stub
         return null;
     }
 
     @Override
-    public PhoenixTransactionalTable getTransactionalTable(PhoenixTransactionContext ctx, Table htable) {
-        //return new OmidTransactionTable(ctx, htable);
+    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo) {
+        // TODO Auto-generated method stub
         return null;
     }
-    
+
     @Override
-    public Cell newDeleteFamilyMarker(byte[] row, byte[] family, long timestamp) {
-        return CellUtil.createCell(row, family, HConstants.EMPTY_BYTE_ARRAY, timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
+    public Class<? extends RegionObserver> getCoprocessor() {
+        // TODO Auto-generated method stub
+        return null;
     }
-    
+
     @Override
-    public Cell newDeleteColumnMarker(byte[] row, byte[] family, byte[] qualifier, long timestamp) {
-        return CellUtil.createCell(row, family, qualifier, timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
+    public Provider getProvider() {
+        return TransactionFactory.Provider.OMID;
     }
 
+    @Override
+    public boolean isUnsupported(Feature feature) {
+        // FIXME: if we initialize a Set with the unsupported features
+        // and check for containment, we run into a test failure
+        // in SetPropertyOnEncodedTableIT.testSpecifyingColumnFamilyForTTLFails()
+        // due to TableProperty.colFamSpecifiedException being null
+        // (though it's set in the constructor). I suspect some
+        // mysterious class loader issue. The below works fine
+        // as a workaround.
+        return (feature == Feature.ALTER_NONTX_TO_TX);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
deleted file mode 100644
index 0874e45..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
+++ /dev/null
@@ -1,380 +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.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompareOperator;
-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.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.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class OmidTransactionTable implements PhoenixTransactionalTable {
-
-    public OmidTransactionTable(PhoenixTransactionContext ctx, Table hTable) {
-        // TODO Auto-generated constructor stub
-    }
-
-    @Override
-    public Result get(Get get) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public void put(Put put) throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void delete(Delete delete) throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public ResultScanner getScanner(Scan scan) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-
-    @Override
-    public Configuration getConfiguration() {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public HTableDescriptor getTableDescriptor() throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public boolean exists(Get get) throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public Result[] get(List<Get> gets) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public ResultScanner getScanner(byte[] family) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public ResultScanner getScanner(byte[] family, byte[] qualifier)
-            throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public void put(List<Put> puts) throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public void delete(List<Delete> deletes) throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    
-
-    @Override
-    public void close() throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public TableName getName() {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public boolean[] existsAll(List<Get> gets) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public void batch(List<? extends Row> actions, Object[] results)
-            throws IOException, InterruptedException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public <R> void batchCallback(List<? extends Row> actions,
-            Object[] results, Callback<R> callback) throws IOException,
-            InterruptedException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-            byte[] value, Put put) throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-            CompareOp compareOp, byte[] value, Put put) throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-            byte[] value, Delete delete) throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-            CompareOp compareOp, byte[] value, Delete delete)
-            throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public void mutateRow(RowMutations rm) throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public Result append(Append append) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public Result increment(Increment increment) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public long incrementColumnValue(byte[] row, byte[] family,
-            byte[] qualifier, long amount) throws IOException {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public long incrementColumnValue(byte[] row, byte[] family,
-            byte[] qualifier, long amount, Durability durability)
-            throws IOException {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public CoprocessorRpcChannel coprocessorService(byte[] row) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public <T extends Service, R> Map<byte[], R> coprocessorService(
-            Class<T> service, byte[] startKey, byte[] endKey,
-            Call<T, R> callable) throws ServiceException, Throwable {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public <T extends Service, R> void coprocessorService(Class<T> service,
-            byte[] startKey, byte[] endKey, Call<T, R> callable,
-            Callback<R> callback) throws ServiceException, Throwable {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public <R extends Message> Map<byte[], R> batchCoprocessorService(
-            MethodDescriptor methodDescriptor, Message request,
-            byte[] startKey, byte[] endKey, R responsePrototype)
-            throws ServiceException, Throwable {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public <R extends Message> void batchCoprocessorService(
-            MethodDescriptor methodDescriptor, Message request,
-            byte[] startKey, byte[] endKey, R responsePrototype,
-            Callback<R> callback) throws ServiceException, Throwable {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-            CompareOp compareOp, byte[] value, RowMutations mutation)
-            throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public int getOperationTimeout() {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public int getRpcTimeout() {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public void setOperationTimeout(int arg0) {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public void setRpcTimeout(int arg0) {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public TableDescriptor getDescriptor() throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public boolean[] exists(List<Get> gets) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Put put)
-            throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value,
-            Delete delete) throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value,
-            RowMutations mutation) throws IOException {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public long getRpcTimeout(TimeUnit unit) {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public long getReadRpcTimeout(TimeUnit unit) {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public int getReadRpcTimeout() {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public void setReadRpcTimeout(int readRpcTimeout) {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public long getWriteRpcTimeout(TimeUnit unit) {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public int getWriteRpcTimeout() {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public void setWriteRpcTimeout(int writeRpcTimeout) {
-        // TODO Auto-generated method stub
-        
-    }
-
-    @Override
-    public long getOperationTimeout(TimeUnit unit) {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionClient.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionClient.java
new file mode 100644
index 0000000..f12f818
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionClient.java
@@ -0,0 +1,23 @@
+/*
+ * 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.phoenix.transaction;
+
+import java.io.Closeable;
+
+public interface PhoenixTransactionClient extends Closeable {
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionContext.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionContext.java
index 2eeef43..05e5c21 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionContext.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionContext.java
@@ -17,21 +17,99 @@
  */
 package org.apache.phoenix.transaction;
 
-import java.io.IOException;
 import java.sql.SQLException;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.twill.zookeeper.ZKClientService;
-import org.slf4j.Logger;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 
 public interface PhoenixTransactionContext {
-
+    public static PhoenixTransactionContext NULL_CONTEXT = new PhoenixTransactionContext() {
+
+        @Override
+        public void begin() throws SQLException {
+        }
+
+        @Override
+        public void commit() throws SQLException {
+        }
+
+        @Override
+        public void abort() throws SQLException {
+        }
+
+        @Override
+        public void checkpoint(boolean hasUncommittedData) throws SQLException {
+        }
+
+        @Override
+        public void commitDDLFence(PTable dataTable) throws SQLException {
+        }
+
+        @Override
+        public void join(PhoenixTransactionContext ctx) {
+        }
+
+        @Override
+        public boolean isTransactionRunning() {
+            return false;
+        }
+
+        @Override
+        public void reset() {
+        }
+
+        @Override
+        public long getTransactionId() {
+            return 0;
+        }
+
+        @Override
+        public long getReadPointer() {
+            return 0;
+        }
+
+        @Override
+        public long getWritePointer() {
+            return 0;
+        }
+
+        @Override
+        public void setVisibilityLevel(PhoenixVisibilityLevel visibilityLevel) {
+        }
+
+        @Override
+        public PhoenixVisibilityLevel getVisibilityLevel() {
+            return null;
+        }
+
+        @Override
+        public byte[] encodeTransaction() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public Provider getProvider() {
+            return null;
+        }
+
+        @Override
+        public PhoenixTransactionContext newTransactionContext(PhoenixTransactionContext contex, boolean subTask) {
+            return NULL_CONTEXT;
+        }
+
+        @Override
+        public void markDMLFence(PTable dataTable) {
+            
+        }
+
+        @Override
+        public Table getTransactionalTable(Table htable, boolean isImmutable) {
+            return null;
+        }
+    };
     /**
      * 
      * Visibility levels needed for checkpointing and  
@@ -51,22 +129,6 @@ public interface PhoenixTransactionContext {
     public static final String READ_NON_TX_DATA = "data.tx.read.pre.existing";
 
     /**
-     * Set the in memory client connection to the transaction manager (for testing purpose)
-     *
-     * @param config
-     */
-    public void setInMemoryTransactionClient(Configuration config);
-
-    /**
-     * Set the client connection to the transaction manager
-     *
-     * @param config
-     * @param props
-     * @param connectionInfo
-     */
-    public ZKClientService setTransactionClient(Configuration config, ReadOnlyProps props, ConnectionInfo connectionInfo);
-
-    /**
      * Starts a transaction
      *
      * @throws SQLException
@@ -86,7 +148,7 @@ public interface PhoenixTransactionContext {
      * @throws SQLException
      */
     public void abort() throws SQLException;
-
+    
     /**
      * Create a checkpoint in a transaction as defined in [TEPHRA-96]
      * @throws SQLException
@@ -102,9 +164,17 @@ public interface PhoenixTransactionContext {
      * @throws InterruptedException
      * @throws TimeoutException
      */
-    public void commitDDLFence(PTable dataTable, Logger logger)
+    public void commitDDLFence(PTable dataTable)
             throws SQLException;
 
+
+    /**
+     * Mark the start of DML go ensure that updates to indexed rows are not
+     * missed.
+     * @param dataTable the table on which DML command is working
+     */
+    public void markDMLFence(PTable dataTable);
+
     /**
      * Augment the current context with ctx modified keys
      *
@@ -123,7 +193,8 @@ public interface PhoenixTransactionContext {
     public void reset();
 
     /**
-     * Returns transaction unique identifier
+     * Returns transaction unique identifier which is also
+     * assumed to be the earliest write pointer.
      */
     public long getTransactionId();
 
@@ -152,42 +223,8 @@ public interface PhoenixTransactionContext {
      */
     public byte[] encodeTransaction() throws SQLException;
 
-    /**
-     * 
-     * @return max transactions per second
-     */
-    public long getMaxTransactionsPerSecond();
+    public Provider getProvider();
+    public PhoenixTransactionContext newTransactionContext(PhoenixTransactionContext contex, boolean subTask);
 
-    /**
-     *
-     * @param version
-     */
-    public boolean isPreExistingVersion(long version);
-
-    /**
-     *
-     * @return the coprocessor
-     */
-    public RegionObserver getCoprocessor();
-
-    /**
-     * 
-     * @return the family delete marker
-     */
-    public byte[] getFamilyDeleteMarker();
-
-    /**
-     * Setup transaction manager's configuration for testing
-     */
-     public void setTxnConfigs(Configuration config, String tmpFolder, int defaultTxnTimeoutSeconds) throws IOException;
-
-    /**
-     * Setup transaction manager for testing
-     */
-    public void setupTxManager(Configuration config, String url) throws SQLException;
-
-    /**
-     * Tear down transaction manager for testing
-     */
-    public void tearDownTxManager();
+    public Table getTransactionalTable(Table htable, boolean isImmutable);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
new file mode 100644
index 0000000..cdc6058
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionProvider.java
@@ -0,0 +1,51 @@
+/*
+ * 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.phoenix.transaction;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+
+public interface PhoenixTransactionProvider {
+    public enum Feature {
+        ALTER_NONTX_TO_TX(SQLExceptionCode.CANNOT_ALTER_TABLE_FROM_NON_TXN_TO_TXNL);
+        
+        private final SQLExceptionCode code;
+        
+        Feature(SQLExceptionCode code) {
+            this.code = code;
+        }
+        
+        public SQLExceptionCode getCode() {
+            return code;
+        }
+    }
+    public PhoenixTransactionContext getTransactionContext(byte[] txnBytes) throws IOException;
+    public PhoenixTransactionContext getTransactionContext(PhoenixConnection connection);
+    
+    public PhoenixTransactionClient getTransactionClient(Configuration config, ConnectionInfo connectionInfo);
+    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo);
+    public Class<? extends RegionObserver> getCoprocessor();
+    
+    public TransactionFactory.Provider getProvider();
+    public boolean isUnsupported(Feature feature);
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionService.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionService.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionService.java
new file mode 100644
index 0000000..10c46e1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionService.java
@@ -0,0 +1,24 @@
+/*
+ * 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.phoenix.transaction;
+
+import java.io.Closeable;
+
+public interface PhoenixTransactionService extends Closeable {
+
+}


[4/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index f05c74b..2759c9f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -70,6 +70,8 @@ import org.apache.phoenix.monitoring.MutationMetricQueue.MutationMetric;
 import org.apache.phoenix.monitoring.MutationMetricQueue.NoOpMutationMetricsQueue;
 import org.apache.phoenix.monitoring.ReadMetricQueue;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
@@ -89,8 +91,8 @@ import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.PhoenixTransactionContext.PhoenixVisibilityLevel;
-import org.apache.phoenix.transaction.PhoenixTransactionalTable;
 import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.LogUtil;
@@ -137,7 +139,7 @@ public class MutationState implements SQLCloseable {
     private boolean isExternalTxContext = false;
     private Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
 
-    final PhoenixTransactionContext phoenixTransactionContext;
+    private PhoenixTransactionContext phoenixTransactionContext = PhoenixTransactionContext.NULL_CONTEXT;
 
     private final MutationMetricQueue mutationMetricQueue;
     private ReadMetricQueue readMetricQueue;
@@ -179,17 +181,13 @@ public class MutationState implements SQLCloseable {
         boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled();
         this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue()
                 : NoOpMutationMetricsQueue.NO_OP_MUTATION_METRICS_QUEUE;
-        if (!subTask) {
-            if (txContext == null) {
-                phoenixTransactionContext = TransactionFactory.getTransactionProvider().getTransactionContext(connection);
-            } else {
-                isExternalTxContext = true;
-                phoenixTransactionContext = TransactionFactory.getTransactionProvider().getTransactionContext(txContext, connection, subTask);
-            }
-        } else {
+        if (subTask) {
             // this code path is only used while running child scans, we can't pass the txContext to child scans
             // as it is not thread safe, so we use the tx member variable
-            phoenixTransactionContext = TransactionFactory.getTransactionProvider().getTransactionContext(txContext, connection, subTask);
+            phoenixTransactionContext = txContext.newTransactionContext(txContext, subTask);
+        } else if (txContext != null) {
+            isExternalTxContext = true;
+            phoenixTransactionContext = txContext.newTransactionContext(txContext, subTask);
         }
     }
 
@@ -232,7 +230,7 @@ public class MutationState implements SQLCloseable {
     public void commitDDLFence(PTable dataTable) throws SQLException {
         if (dataTable.isTransactional()) {
             try {
-                phoenixTransactionContext.commitDDLFence(dataTable, logger);
+                phoenixTransactionContext.commitDDLFence(dataTable);
             } finally {
                 // The client expects a transaction to be in progress on the txContext while the
                 // VisibilityFence.prepareWait() starts a new tx and finishes/aborts it. After it's
@@ -298,14 +296,12 @@ public class MutationState implements SQLCloseable {
     // Though MutationState is not thread safe in general, this method should be because it may
     // be called by TableResultIterator in a multi-threaded manner. Since we do not want to expose
     // the Transaction outside of MutationState, this seems reasonable, as the member variables
-    // would not change as these threads are running.
+    // would not change as these threads are running. We also clone mutationState to ensure that
+    // the transaction context won't change due to a commit when auto commit is true.
     public Table getHTable(PTable table) throws SQLException {
         Table htable = this.getConnection().getQueryServices().getTable(table.getPhysicalName().getBytes());
         if (table.isTransactional() && phoenixTransactionContext.isTransactionRunning()) {
-            PhoenixTransactionalTable phoenixTransactionTable = TransactionUtil.getPhoenixTransactionTable(phoenixTransactionContext, htable, table);
-            // Using cloned mutationState as we may have started a new transaction already
-            // if auto commit is true and we need to use the original one here.
-            htable = phoenixTransactionTable;
+            htable = phoenixTransactionContext.getTransactionalTable(htable, table.isImmutableRows());
         }
         return htable;
     }
@@ -332,13 +328,32 @@ public class MutationState implements SQLCloseable {
         return phoenixTransactionContext.getVisibilityLevel();
     }
 
-    public boolean startTransaction() throws SQLException {
+    public boolean startTransaction(Provider provider) throws SQLException {
+        if (provider == null) {
+            return false;
+        }
+        if (!connection.getQueryServices().getProps().getBoolean(
+                QueryServices.TRANSACTIONS_ENABLED,
+                QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED)) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.CANNOT_START_TXN_IF_TXN_DISABLED)
+                    .build().buildException();
+        }
         if (connection.getSCN() != null) {
             throw new SQLExceptionInfo.Builder(
                     SQLExceptionCode.CANNOT_START_TRANSACTION_WITH_SCN_SET)
                     .build().buildException();
         }
 
+        if (phoenixTransactionContext == PhoenixTransactionContext.NULL_CONTEXT) {
+            phoenixTransactionContext = provider.getTransactionProvider().getTransactionContext(connection);
+        } else {
+            if (provider != phoenixTransactionContext.getProvider()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MIX_TXN_PROVIDERS)
+                        .setMessage(phoenixTransactionContext.getProvider().name() + " and " + provider.name())
+                        .build().buildException();
+            }
+        }
         if (!isTransactionStarted()) {
             // Clear any transactional state in case transaction was ended outside
             // of Phoenix so we don't carry the old transaction state forward. We
@@ -772,7 +787,6 @@ public class MutationState implements SQLCloseable {
     }
     
     private long validateAndGetServerTimestamp(TableRef tableRef, MultiRowMutationState rowKeyToColumnMap) throws SQLException {
-        Long scn = connection.getSCN();
         MetaDataClient client = new MetaDataClient(connection);
         long serverTimeStamp = tableRef.getTimeStamp();
         // If we're auto committing, we've already validated the schema when we got the ColumnResolver,
@@ -1064,6 +1078,7 @@ public class MutationState implements SQLCloseable {
                         if (table.isTransactional()) {
                             // Track tables to which we've sent uncommitted data
                             uncommittedPhysicalNames.add(table.getPhysicalName().getString());
+                            phoenixTransactionContext.markDMLFence(table);
 
                             // If we have indexes, wrap the HTable in a delegate HTable that
                             // will attach the necessary index meta data in the event of a
@@ -1072,7 +1087,7 @@ public class MutationState implements SQLCloseable {
                                 hTable = new MetaDataAwareHTable(hTable, origTableRef);
                             }
 
-                            hTable = TransactionUtil.getPhoenixTransactionTable(phoenixTransactionContext, hTable, table);
+                            hTable = phoenixTransactionContext.getTransactionalTable(hTable, table.isImmutableRows());
                         }
                         
                         numMutations = mutationList.size();
@@ -1189,10 +1204,6 @@ public class MutationState implements SQLCloseable {
         return phoenixTransactionContext.encodeTransaction();
     }
 
-    public static PhoenixTransactionContext decodeTransaction(byte[] txnBytes) throws IOException {
-        return TransactionFactory.getTransactionProvider().getTransactionContext(txnBytes);
-    }
-
     private ServerCache setMetaDataOnMutations(TableRef tableRef, List<? extends Mutation> mutations,
             ImmutableBytesWritable indexMetaDataPtr) throws SQLException {
         PTable table = tableRef.getTable();
@@ -1235,7 +1246,7 @@ public class MutationState implements SQLCloseable {
             mutation.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
             if (attribValue != null) {
                 mutation.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
-                mutation.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, Bytes.toBytes(MetaDataProtocol.PHOENIX_VERSION));
+                mutation.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, Bytes.toBytes(MetaDataProtocol.PHOENIX_VERSION));
                 if (txState.length > 0) {
                     mutation.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
                 }
@@ -1267,7 +1278,7 @@ public class MutationState implements SQLCloseable {
         numRows = 0;
         estimatedSize = 0;
         this.mutations.clear();
-        resetTransactionalState();
+        phoenixTransactionContext = PhoenixTransactionContext.NULL_CONTEXT;
     }
 
     private void resetTransactionalState() {
@@ -1331,13 +1342,18 @@ public class MutationState implements SQLCloseable {
                         }
                     }
                 } finally {
+                    TransactionFactory.Provider provider = phoenixTransactionContext.getProvider();
                     try {
                         resetState();
                     } finally {
                         if (retryCommit) {
-                            startTransaction();
+                            startTransaction(provider);
                             // Add back read fences
                             Set<TableRef> txTableRefs = txMutations.keySet();
+                            for (TableRef tableRef : txTableRefs) {
+                                PTable dataTable = tableRef.getTable();
+                                phoenixTransactionContext.markDMLFence(dataTable);
+                            }
                             try {
                                 // Only retry if an index was added
                                 retryCommit = shouldResubmitTransaction(txTableRefs);
@@ -1442,9 +1458,12 @@ public class MutationState implements SQLCloseable {
             List<TableRef> strippedAliases = Lists.newArrayListWithExpectedSize(mutations.keySet().size());
             while (filteredTableRefs.hasNext()) {
                 TableRef tableRef = filteredTableRefs.next();
+                // REVIEW: unclear if we need this given we start transactions when resolving a table
+                if (tableRef.getTable().isTransactional()) {
+                    startTransaction(tableRef.getTable().getTransactionProvider());
+                }
                 strippedAliases.add(new TableRef(null, tableRef.getTable(), tableRef.getTimeStamp(), tableRef.getLowerBoundTimeStamp(), tableRef.hasDynamicCols()));
             }
-            startTransaction();
             send(strippedAliases.iterator());
             return true;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
index 08f3edc..561aea7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
@@ -31,8 +31,6 @@ import java.util.ListIterator;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
@@ -42,7 +40,6 @@ 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.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -65,8 +62,6 @@ import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.PhoenixTransactionContext.PhoenixVisibilityLevel;
 import org.apache.phoenix.util.PhoenixKeyValueUtil;
-import org.apache.phoenix.transaction.PhoenixTransactionalTable;
-import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -77,9 +72,6 @@ import com.google.common.primitives.Longs;
 
 
 public class PhoenixTxIndexMutationGenerator {
-
-    private static final Log LOG = LogFactory.getLog(PhoenixTxIndexMutationGenerator.class);
-
     private final PhoenixIndexCodec codec;
     private final PhoenixIndexMetaData indexMetaData;
 
@@ -182,7 +174,7 @@ public class PhoenixTxIndexMutationGenerator {
             scan.addColumn(indexMaintainers.get(0).getDataEmptyKeyValueCF(), emptyKeyValueQualifier);
             ScanRanges scanRanges = ScanRanges.create(SchemaUtil.VAR_BINARY_SCHEMA, Collections.singletonList(keys), ScanUtil.SINGLE_COLUMN_SLOT_SPAN, KeyRange.EVERYTHING_RANGE, null, true, -1);
             scanRanges.initializeScan(scan);
-            PhoenixTransactionalTable txTable = TransactionFactory.getTransactionProvider().getTransactionalTable(indexMetaData.getTransactionContext(), htable);
+            Table txTable = indexMetaData.getTransactionContext().getTransactionalTable(htable, isImmutable);
             // For rollback, we need to see all versions, including
             // the last committed version as there may be multiple
             // checkpointed versions.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c5065e0..59c10ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunction;
-import org.apache.phoenix.expression.function.MonthFunction;
-import org.apache.phoenix.expression.function.NowFunction;
-import org.apache.phoenix.expression.function.NthValueFunction;
-import org.apache.phoenix.expression.function.OctetLengthFunction;
-import org.apache.phoenix.expression.function.PercentRankAggregateFunction;
-import org.apache.phoenix.expression.function.PercentileContAggregateFunction;
-import org.apache.phoenix.expression.function.PercentileDiscAggregateFunction;
-import org.apache.phoenix.expression.function.PowerFunction;
-import org.apache.phoenix.expression.function.RTrimFunction;
-import org.apache.phoenix.expression.function.RandomFunction;
-import org.apache.phoenix.expression.function.RegexpReplaceFunction;
-import org.apache.phoenix.expression.function.RegexpSplitFunction;
-import org.apache.phoenix.expression.function.RegexpSubstrFunction;
-import org.apache.phoenix.expression.function.ReverseFunction;
-import org.apache.phoenix.expression.function.RoundDateExpression;
-import org.apache.phoenix.expression.function.RoundDecimalExpression;
-import org.apache.phoenix.expression.function.RoundFunction;
-import org.apache.phoenix.expression.function.RoundMonthExpression;
-import org.apache.phoenix.expression.function.RoundTimestampExpression;
-import org.apache.phoenix.expression.function.RoundWeekExpression;
-import org.apache.phoenix.expression.function.RoundYearExpression;
-import org.apache.phoenix.expression.function.SQLIndexTypeFunction;
-import org.apache.phoenix.expression.function.SQLTableTypeFunction;
-import org.apache.phoenix.expression.function.SQLViewTypeFunction;
-import org.apache.phoenix.expression.function.SecondFunction;
-import org.apache.phoenix.expression.function.SetBitFunction;
-import org.apache.phoenix.expression.function.SetByteFunction;
-import org.apache.phoenix.expression.function.SignFunction;
-import org.apache.phoenix.expression.function.SqlTypeNameFunction;
-import org.apache.phoenix.expression.function.SqrtFunction;
-import org.apache.phoenix.expression.function.StddevPopFunction;
-import org.apache.phoenix.expression.function.StddevSampFunction;
-import org.apache.phoenix.expression.function.StringBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.StringBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.StringBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.StringToArrayFunction;
-import org.apache.phoenix.expression.function.SubstrFunction;
-import org.apache.phoenix.expression.function.SumAggregateFunction;
-import org.apache.phoenix.expression.function.TimezoneOffsetFunction;
-import org.apache.phoenix.expression.function.ToCharFunction;
-import org.apache.phoenix.expression.function.ToDateFunction;
-import org.apache.phoenix.expression.function.ToNumberFunction;
-import org.apache.phoenix.expression.function.ToTimeFunction;
-import org.apache.phoenix.expression.function.ToTimestampFunction;
-import org.apache.phoenix.expression.function.TrimFunction;
-import org.apache.phoenix.expression.function.TruncFunction;
-import org.apache.phoenix.expression.function.UDFExpression;
-import org.apache.phoenix.expression.function.UpperFunction;
-import org.apache.phoenix.expression.function.WeekFunction;
-import org.apache.phoenix.expression.function.YearFunction;
+import org.apache.phoenix.expression.function.*;
 
 import com.google.common.collect.Maps;
 
@@ -298,7 +185,9 @@ public enum ExpressionType {
     LastValuesFunction(LastValuesFunction.class),
     DistinctCountHyperLogLogAggregateFunction(DistinctCountHyperLogLogAggregateFunction.class),
     CollationKeyFunction(CollationKeyFunction.class),
-    ArrayRemoveFunction(ArrayRemoveFunction.class);
+    ArrayRemoveFunction(ArrayRemoveFunction.class),
+    TransactionProviderNameFunction(TransactionProviderNameFunction.class),
+    ;
 
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TransactionProviderNameFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TransactionProviderNameFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TransactionProviderNameFunction.java
new file mode 100644
index 0000000..0117c1f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TransactionProviderNameFunction.java
@@ -0,0 +1,81 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PTinyint;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
+
+
+/**
+ * 
+ * Function used to get the index state name from the serialized byte value
+ * Usage:
+ * IndexStateName('a')
+ * will return 'ACTIVE'
+ * 
+ * 
+ * @since 2.1
+ */
+@BuiltInFunction(name=TransactionProviderNameFunction.NAME, args= {
+    @Argument(allowedTypes= PInteger.class)} )
+public class TransactionProviderNameFunction extends ScalarFunction {
+    public static final String NAME = "TransactionProviderName";
+
+    public TransactionProviderNameFunction() {
+    }
+    
+    public TransactionProviderNameFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression child = children.get(0);
+        if (!child.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        int code = PTinyint.INSTANCE.getCodec().decodeByte(ptr, child.getSortOrder());
+        TransactionFactory.Provider provider = TransactionFactory.Provider.fromCode(code);
+        ptr.set(PVarchar.INSTANCE.toBytes(provider.name()));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PVarchar.INSTANCE;
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 8133e71..2f20af2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -101,7 +101,6 @@ import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.schema.tuple.BaseTuple;
 import org.apache.phoenix.schema.tuple.ValueGetterTuple;
 import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.BitSet;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
@@ -109,6 +108,7 @@ import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TransactionUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 
 import com.google.common.base.Preconditions;
@@ -1068,7 +1068,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             }
         	else if (kv.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()
         			// Since we don't include the index rows in the change set for txn tables, we need to detect row deletes that have transformed by TransactionProcessor
-        			|| (CellUtil.matchingQualifier(kv, TransactionFactory.getTransactionProvider().getTransactionContext().getFamilyDeleteMarker()) && CellUtil.matchingValue(kv, HConstants.EMPTY_BYTE_ARRAY))) {
+        	        || TransactionUtil.isDeleteFamily(kv)) {
         	    nDeleteCF++;
         	}
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
index 94fbd0d..778401e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
@@ -52,7 +52,7 @@ public class IndexMetaDataCacheFactory implements ServerCacheFactory {
                 IndexMaintainer.deserialize(cachePtr, GenericKeyValueBuilder.INSTANCE, useProtoForIndexMaintainer);
         final PhoenixTransactionContext txnContext;
         try {
-            txnContext = txState.length != 0 ? TransactionFactory.getTransactionProvider().getTransactionContext(txState) : null;
+            txnContext = TransactionFactory.getTransactionContext(txState, clientVersion);
         } catch (IOException e) {
             throw new SQLException(e);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
index c36d8b3..04dcbc4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -44,7 +44,6 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
     public static final String INDEX_PROTO_MD = "IdxProtoMD";
     public static final String INDEX_UUID = "IdxUUID";
     public static final String INDEX_MAINTAINERS = "IndexMaintainers";
-    public static final String CLIENT_VERSION = "_ClientVersion";
     public static KeyValueBuilder KV_BUILDER = GenericKeyValueBuilder.INSTANCE;
     
     private byte[] regionStartKey;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaDataBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaDataBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaDataBuilder.java
index ff6e65e..a46f842 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaDataBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaDataBuilder.java
@@ -37,6 +37,7 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
 
 public class PhoenixIndexMetaDataBuilder {
@@ -63,9 +64,9 @@ public class PhoenixIndexMetaDataBuilder {
             boolean useProto = md != null;
             byte[] txState = attributes.get(BaseScannerRegionObserver.TX_STATE);
             final List<IndexMaintainer> indexMaintainers = IndexMaintainer.deserialize(md, useProto);
-            final PhoenixTransactionContext txnContext = TransactionFactory.getTransactionProvider().getTransactionContext(txState);
-            byte[] clientVersionBytes = attributes.get(PhoenixIndexCodec.CLIENT_VERSION);
-            final int clientVersion = clientVersionBytes == null ? IndexMetaDataCache.UNKNOWN_CLIENT_VERSION : Bytes.toInt(clientVersionBytes);
+            byte[] clientVersionBytes = attributes.get(BaseScannerRegionObserver.CLIENT_VERSION);
+            final int clientVersion = clientVersionBytes == null ? ScanUtil.UNKNOWN_CLIENT_VERSION : Bytes.toInt(clientVersionBytes);
+            final PhoenixTransactionContext txnContext = TransactionFactory.getTransactionContext(txState, clientVersion);
             return new IndexMetaDataCache() {
 
                 @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
index bae6300..eab1018 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
@@ -42,7 +42,6 @@ import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.coprocessor.BaseRegionScanner;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.coprocessor.HashJoinRegionScanner;
-import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -63,6 +62,7 @@ import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
@@ -128,12 +128,13 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
       if (localIndexBytes == null) {
         localIndexBytes = scan.getAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD);
       }
+      int clientVersion = ScanUtil.getClientVersion(scan);
       List<IndexMaintainer> indexMaintainers =
           localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes, useProto);
       indexMaintainer = indexMaintainers.get(0);
       viewConstants = IndexUtil.deserializeViewConstantsFromScan(scan);
       byte[] txState = scan.getAttribute(BaseScannerRegionObserver.TX_STATE);
-      tx = MutationState.decodeTransaction(txState);
+      tx = TransactionFactory.getTransactionContext(txState, clientVersion);
     }
 
     final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 885c86a..ad187e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -48,6 +48,7 @@ import org.apache.phoenix.expression.function.SQLIndexTypeFunction;
 import org.apache.phoenix.expression.function.SQLTableTypeFunction;
 import org.apache.phoenix.expression.function.SQLViewTypeFunction;
 import org.apache.phoenix.expression.function.SqlTypeNameFunction;
+import org.apache.phoenix.expression.function.TransactionProviderNameFunction;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.iterate.DelegateResultIterator;
@@ -298,6 +299,9 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String TRANSACTIONAL = "TRANSACTIONAL";
     public static final byte[] TRANSACTIONAL_BYTES = Bytes.toBytes(TRANSACTIONAL);
 
+    public static final String TRANSACTION_PROVIDER = "TRANSACTION_PROVIDER";
+    public static final byte[] TRANSACTION_PROVIDER_BYTES = Bytes.toBytes(TRANSACTION_PROVIDER);
+
     public static final String UPDATE_CACHE_FREQUENCY = "UPDATE_CACHE_FREQUENCY";
     public static final byte[] UPDATE_CACHE_FREQUENCY_BYTES = Bytes.toBytes(UPDATE_CACHE_FREQUENCY);
 
@@ -1133,9 +1137,10 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
                     VIEW_STATEMENT + "," +
                     SQLViewTypeFunction.NAME + "(" + VIEW_TYPE + ") AS " + VIEW_TYPE + "," +
                     SQLIndexTypeFunction.NAME + "(" + INDEX_TYPE + ") AS " + INDEX_TYPE + "," +
-                    TRANSACTIONAL + "," +
+                    TRANSACTION_PROVIDER + " IS NOT NULL AS " + TRANSACTIONAL + "," +
                     IS_NAMESPACE_MAPPED + "," +
-                    GUIDE_POSTS_WIDTH +
+                    GUIDE_POSTS_WIDTH + "," +
+                    TransactionProviderNameFunction.NAME + "(" + TRANSACTION_PROVIDER + ") AS TRANSACTION_PROVIDER" +
                     " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
                     " where " + COLUMN_NAME + " is null" +
                     " and " + COLUMN_FAMILY + " is null" +
@@ -1175,7 +1180,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
                     "'' " + INDEX_TYPE + "," +
                     "CAST(null AS BOOLEAN) " + TRANSACTIONAL + "," +
                     "CAST(null AS BOOLEAN) " + IS_NAMESPACE_MAPPED + "," +
-                    "CAST(null AS BIGINT) " + GUIDE_POSTS_WIDTH + "\n");
+                    "CAST(null AS BIGINT) " + GUIDE_POSTS_WIDTH + "," +
+                    "CAST(null AS VARCHAR) " + TRANSACTION_PROVIDER + "\n");
             buf.append(
                     " from " + SYSTEM_SEQUENCE + "\n");
             StringBuilder whereClause = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 76b5a1a..e199e61 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -405,7 +405,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                                 MutationState state = connection.getMutationState();
                                 MutationPlan plan = stmt.compilePlan(PhoenixStatement.this, Sequence.ValueOp.VALIDATE_SEQUENCE);
                                 if (plan.getTargetRef() != null && plan.getTargetRef().getTable() != null && plan.getTargetRef().getTable().isTransactional()) {
-                                    state.startTransaction();
+                                    state.startTransaction(plan.getTargetRef().getTable().getTransactionProvider());
                                 }
                                 Iterator<TableRef> tableRefs = plan.getSourceRefs().iterator();
                                 state.sendUncommitted(tableRefs);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
index 1ad1ce5..67ec62b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
@@ -116,7 +116,7 @@ public class PhoenixIndexPartialBuildMapper extends TableMapper<ImmutableBytesWr
                         put = new Put(CellUtil.cloneRow(cell));
                         put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                         put.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
-                        put.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, clientVersion);
+                        put.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersion);
                         put.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES, BaseScannerRegionObserver.REPLAY_ONLY_INDEX_WRITES);
                         mutations.add(put);
                     }
@@ -126,7 +126,7 @@ public class PhoenixIndexPartialBuildMapper extends TableMapper<ImmutableBytesWr
                         del = new Delete(CellUtil.cloneRow(cell));
                         del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                         del.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
-                        del.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, clientVersion);
+                        del.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersion);
                         del.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES, BaseScannerRegionObserver.REPLAY_ONLY_INDEX_WRITES);
                         mutations.add(del);
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index f068770..4822cf1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -49,6 +49,8 @@ import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
+import org.apache.phoenix.transaction.PhoenixTransactionClient;
+import org.apache.phoenix.transaction.TransactionFactory;
 
 
 public interface ConnectionQueryServices extends QueryServices, MetaDataMutated {
@@ -153,4 +155,6 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public User getUser();
 
     public QueryLoggerDisruptor getQueryDisruptor();
+    
+    public PhoenixTransactionClient initTransactionClient(TransactionFactory.Provider provider);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 138bf7c..dc9ef04 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -122,6 +122,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
@@ -144,10 +145,10 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
 import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
-import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
 import org.apache.phoenix.coprocessor.ScanRegionObserver;
 import org.apache.phoenix.coprocessor.SequenceRegionObserver;
 import org.apache.phoenix.coprocessor.ServerCachingEndpointImpl;
+import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
@@ -232,8 +233,10 @@ import org.apache.phoenix.schema.types.PTinyint;
 import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.PhoenixTransactionClient;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.ConfigUtil;
@@ -270,7 +273,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static final int DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS = 1000;
     private static final int TTL_FOR_MUTEX = 15 * 60; // 15min 
     protected final Configuration config;
-    private final ConnectionInfo connectionInfo;
+    protected final ConnectionInfo connectionInfo;
     // Copy of config.getProps(), but read-only to prevent synchronization that we
     // don't need.
     private final ReadOnlyProps props;
@@ -312,6 +315,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // List of queues instead of a single queue to provide reduced contention via lock striping
     private final List<LinkedBlockingQueue<WeakReference<PhoenixConnection>>> connectionQueues;
     private ScheduledExecutorService renewLeaseExecutor;
+    private PhoenixTransactionClient[] txClients = new PhoenixTransactionClient[TransactionFactory.Provider.values().length];;
     /*
      * We can have multiple instances of ConnectionQueryServices. By making the thread factory
      * static, renew lease thread names will be unique across them.
@@ -416,23 +420,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     }
 
-    private void initTxServiceClient() {
-        txZKClientService = TransactionFactory.getTransactionProvider().getTransactionContext().setTransactionClient(config, props, connectionInfo);
-    }
-
     private void openConnection() throws SQLException {
         try {
-            boolean transactionsEnabled = props.getBoolean(
-                    QueryServices.TRANSACTIONS_ENABLED,
-                    QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
             GLOBAL_HCONNECTIONS_COUNTER.increment();
             logger.info("HConnection established. Stacktrace for informational purposes: " + connection + " " +  LogUtil.getCallerStackTrace());
-            // only initialize the tx service client if needed and if we succeeded in getting a connection
-            // to HBase
-            if (transactionsEnabled) {
-                initTxServiceClient();
-            }
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
             .setRootCause(e).build().buildException();
@@ -523,7 +515,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             renewLeaseExecutor.shutdownNow();
                         }
                         // shut down the tx client service if we created one to support transactions
-                        if (this.txZKClientService != null) this.txZKClientService.stopAndWait();
+                        for (PhoenixTransactionClient client : txClients) {
+                            if (client != null) {
+                                client.close();
+                            }
+                        }
                     }
                 } catch (IOException e) {
                     if (sqlE == null) {
@@ -868,9 +864,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if(!newDesc.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 builder.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
+            // For ALTER TABLE
+            boolean nonTxToTx = Boolean.TRUE.equals(tableProps.get(PhoenixTransactionContext.READ_NON_TX_DATA));
             boolean isTransactional =
-                    Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
-                    Boolean.TRUE.equals(tableProps.get(PhoenixTransactionContext.READ_NON_TX_DATA)); // For ALTER TABLE
+                    Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) || nonTxToTx;
             // TODO: better encapsulation for this
             // Since indexes can't have indexes, don't install our indexing coprocessor for indexes.
             // Also don't install on the SYSTEM.CATALOG and SYSTEM.STATS table because we use
@@ -933,13 +930,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
 
             if (isTransactional) {
-                if(!newDesc.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
-                    builder.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
+                TransactionFactory.Provider provider = (TransactionFactory.Provider)TableProperty.TRANSACTION_PROVIDER.getValue(tableProps);
+                if (provider == null) {
+                    String providerValue = this.props.get(QueryServices.DEFAULT_TRANSACTION_PROVIDER_ATTRIB, QueryServicesOptions.DEFAULT_TRANSACTION_PROVIDER);
+                    provider = (TransactionFactory.Provider)TableProperty.TRANSACTION_PROVIDER.getValue(providerValue);
+                }
+                Class<? extends RegionObserver> coprocessorClass = provider.getTransactionProvider().getCoprocessor();
+                if (!newDesc.hasCoprocessor(coprocessorClass.getName())) {
+                    builder.addCoprocessor(coprocessorClass.getName(), null, priority - 10, null);
                 }
             } else {
-                // If exception on alter table to transition back to non transactional
-                if(newDesc.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
-                    builder.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
+                // Remove all potential transactional coprocessors
+                for (TransactionFactory.Provider provider : TransactionFactory.Provider.values()) {
+                    Class<? extends RegionObserver> coprocessorClass = provider.getTransactionProvider().getCoprocessor();
+                    if (coprocessorClass != null && newDesc.hasCoprocessor(coprocessorClass.getName())) {
+                        builder.removeCoprocessor(coprocessorClass.getName());
+                    }
                 }
             }
         } catch (IOException e) {
@@ -1134,7 +1140,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     // If we think we're creating a non transactional table when it's already
                     // transactional, don't allow.
-                    if (existingDesc.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
+                    if (existingDesc.hasCoprocessor(TephraTransactionalProcessor.class.getName())) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX)
                         .setSchemaName(SchemaUtil.getSchemaNameFromFullName(physicalTableName))
                         .setTableName(SchemaUtil.getTableNameFromFullName(physicalTableName)).build().buildException();
@@ -2929,6 +2935,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 + PBoolean.INSTANCE.getSqlTypeName());
                     addParentToChildLinks(metaConnection);
                 }
+                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0) {
+                    metaConnection = addColumnsIfNotExists(
+                        metaConnection,
+                        PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                        MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0,
+                        PhoenixDatabaseMetaData.TRANSACTION_PROVIDER + " "
+                                + PTinyint.INSTANCE.getSqlTypeName());
+                }
             }
 
 
@@ -4545,4 +4559,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public QueryLoggerDisruptor getQueryDisruptor() {
         return this.queryDisruptor;
     }
+
+    @Override
+    public synchronized PhoenixTransactionClient initTransactionClient(Provider provider) {
+        PhoenixTransactionClient client = txClients[provider.ordinal()];
+        if (client == null) {
+            client = txClients[provider.ordinal()] = provider.getTransactionProvider().getTransactionClient(config, connectionInfo);
+        }
+        return client;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 7694fd0..4ca6c26 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -84,7 +84,8 @@ import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
-import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.PhoenixTransactionClient;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.JDBCUtil;
 import org.apache.phoenix.util.MetaDataUtil;
@@ -143,7 +144,6 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         // Without making a copy of the configuration we cons up, we lose some of our properties
         // on the server side during testing.
         this.config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration(config);
-        TransactionFactory.getTransactionProvider().getTransactionContext().setInMemoryTransactionClient(config);
         this.guidePostsCache = new GuidePostsCache(this, config);
     }
 
@@ -686,4 +686,9 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     public QueryLoggerDisruptor getQueryDisruptor() {
         return null;
     }
+    
+    @Override
+    public PhoenixTransactionClient initTransactionClient(Provider provider) {
+        return null; // Client is not necessary
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 2ab73b2..a839688 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -50,6 +50,8 @@ import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
+import org.apache.phoenix.transaction.PhoenixTransactionClient;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 
 
 public class DelegateConnectionQueryServices extends DelegateQueryServices implements ConnectionQueryServices {
@@ -365,5 +367,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
         return getDelegate().getQueryDisruptor();
     }
     
-    
+    @Override
+    public PhoenixTransactionClient initTransactionClient(Provider provider) {
+        return getDelegate().initTransactionClient(provider);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index c30d3d4..67cf1b4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -18,7 +18,112 @@
 package org.apache.phoenix.query;
 
 
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.*;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_VIEW_REFERENCED;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.JAR_PATH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_STATS_UPDATE_TIME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_ARGS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_PREC_RADIX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REF_GENERATION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REMARKS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SELF_REFERENCING_COL_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SOURCE_DATA_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATA_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATETIME_SUB;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.START_WITH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BIND_PARAMETERS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLIENT_IP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.EXCEPTION_TRACE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.EXPLAIN_PLAN;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GLOBAL_SCAN_DETAILS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NO_OF_RESULTS_ITERATED;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.QUERY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.QUERY_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.QUERY_STATUS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCAN_METRICS_JSON;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.START_TIME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TOTAL_EXECUTION_TIME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE;
 
 import java.math.BigDecimal;
 
@@ -236,6 +341,7 @@ public interface QueryConstants {
             ENCODING_SCHEME + " TINYINT, " +
             COLUMN_QUALIFIER_COUNTER + " INTEGER, " +
             USE_STATS_FOR_PARALLELIZATION + " BOOLEAN, " +
+            TRANSACTION_PROVIDER + " TINYINT, " +
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
             + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
             HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 0403ff4..66ec076 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -205,6 +205,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String DEFAULT_KEEP_DELETED_CELLS_ATTRIB = "phoenix.table.default.keep.deleted.cells";
     public static final String DEFAULT_STORE_NULLS_ATTRIB = "phoenix.table.default.store.nulls";
     public static final String DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB = "phoenix.table.istransactional.default";
+    public static final String DEFAULT_TRANSACTION_PROVIDER_ATTRIB = "phoenix.table.transaction.provider.default";
     public static final String GLOBAL_METRICS_ENABLED = "phoenix.query.global.metrics.enabled";
     
     // Transaction related configs

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 427796f..7f34446 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -114,6 +114,7 @@ import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableRefFactory;
 import org.apache.phoenix.trace.util.Tracing;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 
@@ -262,6 +263,7 @@ public class QueryServicesOptions {
     // We'll also need this for transactions to work correctly
     public static final boolean DEFAULT_AUTO_COMMIT = false;
     public static final boolean DEFAULT_TABLE_ISTRANSACTIONAL = false;
+    public static final String DEFAULT_TRANSACTION_PROVIDER = TransactionFactory.Provider.getDefault().name();
     public static final boolean DEFAULT_TRANSACTIONS_ENABLED = false;
     public static final boolean DEFAULT_IS_GLOBAL_METRICS_ENABLED = true;
 
@@ -554,22 +556,22 @@ public class QueryServicesOptions {
         return set(GROUPBY_SPILL_FILES_ATTRIB, num);
     }
 
-    private QueryServicesOptions set(String name, boolean value) {
+    QueryServicesOptions set(String name, boolean value) {
         config.set(name, Boolean.toString(value));
         return this;
     }
 
-    private QueryServicesOptions set(String name, int value) {
+    QueryServicesOptions set(String name, int value) {
         config.set(name, Integer.toString(value));
         return this;
     }
 
-    private QueryServicesOptions set(String name, String value) {
+    QueryServicesOptions set(String name, String value) {
         config.set(name, value);
         return this;
     }
 
-    private QueryServicesOptions set(String name, long value) {
+    QueryServicesOptions set(String name, long value) {
         config.set(name, Long.toString(value));
         return this;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 8f15c5e..d1b8f1e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.transaction.TransactionFactory;
 
 public class DelegateTable implements PTable {
     @Override
@@ -237,7 +238,12 @@ public class DelegateTable implements PTable {
     }
 
     @Override
-    public boolean isTransactional() {
+    public TransactionFactory.Provider getTransactionProvider() {
+        return delegate.getTransactionProvider();
+    }
+
+    @Override
+    public final boolean isTransactional() {
         return delegate.isTransactional();
     }
 


[2/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
deleted file mode 100644
index deceac6..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
+++ /dev/null
@@ -1,798 +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.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompareOperator;
-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.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-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.TableBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.filter.CompareFilter;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public interface PhoenixTransactionalTable extends Table {
-    /**
-     * Gets the fully qualified table name instance of this table.
-     */
-    TableName getName();
-
-    /**
-     * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
-     * <p>
-     * The reference returned is not a copy, so any change made to it will
-     * affect this instance.
-     */
-    Configuration getConfiguration();
-
-    /**
-     * Gets the {@link org.apache.hadoop.hbase.HTableDescriptor table descriptor} for this table.
-     * @throws java.io.IOException if a remote or network exception occurs.
-     * @deprecated since 2.0 version and will be removed in 3.0 version.
-     *             use {@link #getDescriptor()}
-     */
-    @Deprecated
-    HTableDescriptor getTableDescriptor() throws IOException;
-
-    /**
-     * Gets the {@link org.apache.hadoop.hbase.client.TableDescriptor table descriptor} for this table.
-     * @throws java.io.IOException if a remote or network exception occurs.
-     */
-    TableDescriptor getDescriptor() throws IOException;
-
-    /**
-     * Test for the existence of columns in the table, as specified by the Get.
-     * <p>
-     *
-     * This will return true if the Get matches one or more keys, false if not.
-     * <p>
-     *
-     * This is a server-side call so it prevents any data from being transfered to
-     * the client.
-     *
-     * @param get the Get
-     * @return true if the specified Get matches one or more keys, false if not
-     * @throws IOException e
-     */
-    boolean exists(Get get) throws IOException;
-
-    /**
-     * Test for the existence of columns in the table, as specified by the Gets.
-     * <p>
-     *
-     * This will return an array of booleans. Each value will be true if the related Get matches
-     * one or more keys, false if not.
-     * <p>
-     *
-     * This is a server-side call so it prevents any data from being transferred to
-     * the client.
-     *
-     * @param gets the Gets
-     * @return Array of boolean.  True if the specified Get matches one or more keys, false if not.
-     * @throws IOException e
-     */
-    boolean[] exists(List<Get> gets) throws IOException;
-
-    /**
-     * Test for the existence of columns in the table, as specified by the Gets.
-     * This will return an array of booleans. Each value will be true if the related Get matches
-     * one or more keys, false if not.
-     * This is a server-side call so it prevents any data from being transferred to
-     * the client.
-     *
-     * @param gets the Gets
-     * @return Array of boolean.  True if the specified Get matches one or more keys, false if not.
-     * @throws IOException e
-     * @deprecated since 2.0 version and will be removed in 3.0 version.
-     *             use {@link #exists(List)}
-     */
-    @Deprecated
-    default boolean[] existsAll(List<Get> gets) throws IOException {
-      return exists(gets);
-    }
-
-    /**
-     * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations.
-     * The ordering of execution of the actions is not defined. Meaning if you do a Put and a
-     * Get in the same {@link #batch} call, you will not necessarily be
-     * guaranteed that the Get returns what the Put had put.
-     *
-     * @param actions list of Get, Put, Delete, Increment, Append, RowMutations.
-     * @param results Empty Object[], same size as actions. Provides access to partial
-     *                results, in case an exception is thrown. A null in the result array means that
-     *                the call for that action failed, even after retries. The order of the objects
-     *                in the results array corresponds to the order of actions in the request list.
-     * @throws IOException
-     * @since 0.90.0
-     */
-    void batch(final List<? extends Row> actions, final Object[] results) throws IOException,
-      InterruptedException;
-
-    /**
-     * Same as {@link #batch(List, Object[])}, but with a callback.
-     * @since 0.96.0
-     */
-    <R> void batchCallback(
-      final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback
-    ) throws IOException, InterruptedException;
-
-    /**
-     * Extracts certain cells from a given row.
-     * @param get The object that specifies what data to fetch and from which row.
-     * @return The data coming from the specified row, if it exists.  If the row
-     * specified doesn't exist, the {@link Result} instance returned won't
-     * contain any {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    Result get(Get get) throws IOException;
-
-    /**
-     * Extracts specified cells from the given rows, as a batch.
-     *
-     * @param gets The objects that specify what data to fetch and from which rows.
-     * @return The data coming from the specified rows, if it exists.  If the row specified doesn't
-     * exist, the {@link Result} instance returned won't contain any {@link
-     * org.apache.hadoop.hbase.Cell}s, as indicated by {@link Result#isEmpty()}. If there are any
-     * failures even after retries, there will be a <code>null</code> in the results' array for those
-     * Gets, AND an exception will be thrown. The ordering of the Result array corresponds to the order
-     * of the list of passed in Gets.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.90.0
-     * @apiNote {@link #put(List)} runs pre-flight validations on the input list on client.
-     * Currently {@link #get(List)} doesn't run any validations on the client-side, currently there
-     * is no need, but this may change in the future. An
-     * {@link IllegalArgumentException} will be thrown in this case.
-     */
-    Result[] get(List<Get> gets) throws IOException;
-
-    /**
-     * Returns a scanner on the current table as specified by the {@link Scan}
-     * object.
-     * Note that the passed {@link Scan}'s start row and caching properties
-     * maybe changed.
-     *
-     * @param scan A configured {@link Scan} object.
-     * @return A scanner.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    ResultScanner getScanner(Scan scan) throws IOException;
-
-    /**
-     * Gets a scanner on the current table for the given family.
-     *
-     * @param family The column family to scan.
-     * @return A scanner.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    ResultScanner getScanner(byte[] family) throws IOException;
-
-    /**
-     * Gets a scanner on the current table for the given family and qualifier.
-     *
-     * @param family The column family to scan.
-     * @param qualifier The column qualifier to scan.
-     * @return A scanner.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException;
-
-
-    /**
-     * Puts some data in the table.
-     *
-     * @param put The data to put.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    void put(Put put) throws IOException;
-
-    /**
-     * Batch puts the specified data into the table.
-     * <p>
-     * This can be used for group commit, or for submitting user defined batches. Before sending
-     * a batch of mutations to the server, the client runs a few validations on the input list. If an
-     * error is found, for example, a mutation was supplied but was missing it's column an
-     * {@link IllegalArgumentException} will be thrown and no mutations will be applied. If there
-     * are any failures even after retries, a {@link RetriesExhaustedWithDetailsException} will be
-     * thrown. RetriesExhaustedWithDetailsException contains lists of failed mutations and
-     * corresponding remote exceptions. The ordering of mutations and exceptions in the
-     * encapsulating exception corresponds to the order of the input list of Put requests.
-     *
-     * @param puts The list of mutations to apply.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    void put(List<Put> puts) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected
-     * value. If it does, it adds the put.  If the passed value is null, the check
-     * is for the lack of column (ie: non-existance)
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param value the expected value
-     * @param put data to put if check succeeds
-     * @throws IOException e
-     * @return true if the new put was executed, false otherwise
-     */
-    boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      byte[] value, Put put) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected
-     * value. If it does, it adds the put.  If the passed value is null, the check
-     * is for the lack of column (ie: non-existence)
-     *
-     * The expected value argument of this call is on the left and the current
-     * value of the cell is on the right side of the comparison operator.
-     *
-     * Ie. eg. GREATER operator means expected value > existing <=> add the put.
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param compareOp comparison operator to use
-     * @param value the expected value
-     * @param put data to put if check succeeds
-     * @throws IOException e
-     * @return true if the new put was executed, false otherwise
-     * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-     *  {@link #checkAndPut(byte[], byte[], byte[], CompareOperator, byte[], Put)}}
-     */
-    @Deprecated
-    boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected
-     * value. If it does, it adds the put.  If the passed value is null, the check
-     * is for the lack of column (ie: non-existence)
-     *
-     * The expected value argument of this call is on the left and the current
-     * value of the cell is on the right side of the comparison operator.
-     *
-     * Ie. eg. GREATER operator means expected value > existing <=> add the put.
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param op comparison operator to use
-     * @param value the expected value
-     * @param put data to put if check succeeds
-     * @throws IOException e
-     * @return true if the new put was executed, false otherwise
-     */
-    boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-                        CompareOperator op, byte[] value, Put put) throws IOException;
-
-    /**
-     * Deletes the specified cells/row.
-     *
-     * @param delete The object that specifies what to delete.
-     * @throws IOException if a remote or network exception occurs.
-     * @since 0.20.0
-     */
-    void delete(Delete delete) throws IOException;
-
-    /**
-     * Batch Deletes the specified cells/rows from the table.
-     * <p>
-     * If a specified row does not exist, {@link Delete} will report as though sucessful
-     * delete; no exception will be thrown. If there are any failures even after retries,
-     * a * {@link RetriesExhaustedWithDetailsException} will be thrown.
-     * RetriesExhaustedWithDetailsException contains lists of failed {@link Delete}s and
-     * corresponding remote exceptions.
-     *
-     * @param deletes List of things to delete. The input list gets modified by this
-     * method. All successfully applied {@link Delete}s in the list are removed (in particular it
-     * gets re-ordered, so the order in which the elements are inserted in the list gives no
-     * guarantee as to the order in which the {@link Delete}s are executed).
-     * @throws IOException if a remote or network exception occurs. In that case
-     * the {@code deletes} argument will contain the {@link Delete} instances
-     * that have not be successfully applied.
-     * @since 0.20.1
-     * @apiNote In 3.0.0 version, the input list {@code deletes} will no longer be modified. Also,
-     * {@link #put(List)} runs pre-flight validations on the input list on client. Currently
-     * {@link #delete(List)} doesn't run validations on the client, there is no need currently,
-     * but this may change in the future. An * {@link IllegalArgumentException} will be thrown
-     * in this case.
-     */
-    void delete(List<Delete> deletes) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected
-     * value. If it does, it adds the delete.  If the passed value is null, the
-     * check is for the lack of column (ie: non-existance)
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param value the expected value
-     * @param delete data to delete if check succeeds
-     * @throws IOException e
-     * @return true if the new delete was executed, false otherwise
-     */
-    boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-      byte[] value, Delete delete) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected
-     * value. If it does, it adds the delete.  If the passed value is null, the
-     * check is for the lack of column (ie: non-existence)
-     *
-     * The expected value argument of this call is on the left and the current
-     * value of the cell is on the right side of the comparison operator.
-     *
-     * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param compareOp comparison operator to use
-     * @param value the expected value
-     * @param delete data to delete if check succeeds
-     * @throws IOException e
-     * @return true if the new delete was executed, false otherwise
-     * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-     *  {@link #checkAndDelete(byte[], byte[], byte[], byte[], Delete)}
-     */
-    @Deprecated
-    boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-      CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected
-     * value. If it does, it adds the delete.  If the passed value is null, the
-     * check is for the lack of column (ie: non-existence)
-     *
-     * The expected value argument of this call is on the left and the current
-     * value of the cell is on the right side of the comparison operator.
-     *
-     * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param op comparison operator to use
-     * @param value the expected value
-     * @param delete data to delete if check succeeds
-     * @throws IOException e
-     * @return true if the new delete was executed, false otherwise
-     */
-    boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-                           CompareOperator op, byte[] value, Delete delete) throws IOException;
-
-    /**
-     * Performs multiple mutations atomically on a single row. Currently
-     * {@link Put} and {@link Delete} are supported.
-     *
-     * @param rm object that specifies the set of mutations to perform atomically
-     * @throws IOException
-     */
-    void mutateRow(final RowMutations rm) throws IOException;
-
-    /**
-     * Appends values to one or more columns within a single row.
-     * <p>
-     * This operation guaranteed atomicity to readers. Appends are done
-     * under a single row lock, so write operations to a row are synchronized, and
-     * readers are guaranteed to see this operation fully completed.
-     *
-     * @param append object that specifies the columns and amounts to be used
-     *                  for the increment operations
-     * @throws IOException e
-     * @return values of columns after the append operation (maybe null)
-     */
-    Result append(final Append append) throws IOException;
-
-    /**
-     * Increments one or more columns within a single row.
-     * <p>
-     * This operation ensures atomicity to readers. Increments are done
-     * under a single row lock, so write operations to a row are synchronized, and
-     * readers are guaranteed to see this operation fully completed.
-     *
-     * @param increment object that specifies the columns and amounts to be used
-     *                  for the increment operations
-     * @throws IOException e
-     * @return values of columns after the increment
-     */
-    Result increment(final Increment increment) throws IOException;
-
-    /**
-     * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
-     * <p>
-     * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
-     * @param row The row that contains the cell to increment.
-     * @param family The column family of the cell to increment.
-     * @param qualifier The column qualifier of the cell to increment.
-     * @param amount The amount to increment the cell with (or decrement, if the
-     * amount is negative).
-     * @return The new value, post increment.
-     * @throws IOException if a remote or network exception occurs.
-     */
-    long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount) throws IOException;
-
-    /**
-     * Atomically increments a column value. If the column value already exists
-     * and is not a big-endian long, this could throw an exception. If the column
-     * value does not yet exist it is initialized to <code>amount</code> and
-     * written to the specified column.
-     *
-     * <p>Setting durability to {@link Durability#SKIP_WAL} means that in a fail
-     * scenario you will lose any increments that have not been flushed.
-     * @param row The row that contains the cell to increment.
-     * @param family The column family of the cell to increment.
-     * @param qualifier The column qualifier of the cell to increment.
-     * @param amount The amount to increment the cell with (or decrement, if the
-     * amount is negative).
-     * @param durability The persistence guarantee for this increment.
-     * @return The new value, post increment.
-     * @throws IOException if a remote or network exception occurs.
-     */
-    long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount, Durability durability) throws IOException;
-
-    /**
-     * Releases any resources held or pending changes in internal buffers.
-     *
-     * @throws IOException if a remote or network exception occurs.
-     */
-    @Override
-    void close() throws IOException;
-
-    /**
-     * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the
-     * table region containing the specified row.  The row given does not actually have
-     * to exist.  Whichever region would contain the row based on start and end keys will
-     * be used.  Note that the {@code row} parameter is also not passed to the
-     * coprocessor handler registered for this protocol, unless the {@code row}
-     * is separately passed as an argument in the service request.  The parameter
-     * here is only used to locate the region used to handle the call.
-     *
-     * <p>
-     * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
-     * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
-     * </p>
-     *
-     * <div style="background-color: #cccccc; padding: 2px">
-     * <blockquote><pre>
-     * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);
-     * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
-     * MyCallRequest request = MyCallRequest.newBuilder()
-     *     ...
-     *     .build();
-     * MyCallResponse response = service.myCall(null, request);
-     * </pre></blockquote></div>
-     *
-     * @param row The row key used to identify the remote region location
-     * @return A CoprocessorRpcChannel instance
-     */
-    CoprocessorRpcChannel coprocessorService(byte[] row);
-
-    /**
-     * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
-     * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), and
-     * invokes the passed {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
-     * with each {@link com.google.protobuf.Service} instance.
-     *
-     * @param service the protocol buffer {@code Service} implementation to call
-     * @param startKey start region selection with region containing this row.  If {@code null}, the
-     * selection will start with the first table region.
-     * @param endKey select regions up to and including the region containing this row. If {@code
-     * null}, selection will continue through the last table region.
-     * @param callable this instance's {@link org.apache.hadoop.hbase.client.coprocessor.Batch
-     * .Call#call}
-     * method will be invoked once per table region, using the {@link com.google.protobuf.Service}
-     * instance connected to that region.
-     * @param <T> the {@link com.google.protobuf.Service} subclass to connect to
-     * @param <R> Return type for the {@code callable} parameter's {@link
-     * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
-     * @return a map of result values keyed by region name
-     */
-    <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
-      byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
-      throws ServiceException, Throwable;
-
-    /**
-     * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
-     * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), and
-     * invokes the passed {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
-     * with each {@link Service} instance.
-     *
-     * <p> The given {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],
-     * byte[], Object)} method will be called with the return value from each region's {@link
-     * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation. </p>
-     *
-     * @param service the protocol buffer {@code Service} implementation to call
-     * @param startKey start region selection with region containing this row.  If {@code null}, the
-     * selection will start with the first table region.
-     * @param endKey select regions up to and including the region containing this row. If {@code
-     * null}, selection will continue through the last table region.
-     * @param callable this instance's {@link org.apache.hadoop.hbase.client.coprocessor.Batch
-     * .Call#call}
-     * method will be invoked once per table region, using the {@link Service} instance connected to
-     * that region.
-     * @param callback
-     * @param <T> the {@link Service} subclass to connect to
-     * @param <R> Return type for the {@code callable} parameter's {@link
-     * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
-     */
-    <T extends Service, R> void coprocessorService(final Class<T> service,
-      byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
-      final Batch.Callback<R> callback) throws ServiceException, Throwable;
-
-    /**
-     * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
-     * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all
-     * the invocations to the same region server will be batched into one call. The coprocessor
-     * service is invoked according to the service instance, method name and parameters.
-     *
-     * @param methodDescriptor
-     *          the descriptor for the protobuf service method to call.
-     * @param request
-     *          the method call parameters
-     * @param startKey
-     *          start region selection with region containing this row. If {@code null}, the
-     *          selection will start with the first table region.
-     * @param endKey
-     *          select regions up to and including the region containing this row. If {@code null},
-     *          selection will continue through the last table region.
-     * @param responsePrototype
-     *          the proto type of the response of the method in Service.
-     * @param <R>
-     *          the response type for the coprocessor Service method
-     * @throws ServiceException
-     * @throws Throwable
-     * @return a map of result values keyed by region name
-     */
-    <R extends Message> Map<byte[], R> batchCoprocessorService(
-      Descriptors.MethodDescriptor methodDescriptor, Message request,
-      byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable;
-
-    /**
-     * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
-     * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all
-     * the invocations to the same region server will be batched into one call. The coprocessor
-     * service is invoked according to the service instance, method name and parameters.
-     *
-     * <p>
-     * The given
-     * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
-     * method will be called with the return value from each region's invocation.
-     * </p>
-     *
-     * @param methodDescriptor
-     *          the descriptor for the protobuf service method to call.
-     * @param request
-     *          the method call parameters
-     * @param startKey
-     *          start region selection with region containing this row. If {@code null}, the
-     *          selection will start with the first table region.
-     * @param endKey
-     *          select regions up to and including the region containing this row. If {@code null},
-     *          selection will continue through the last table region.
-     * @param responsePrototype
-     *          the proto type of the response of the method in Service.
-     * @param callback
-     *          callback to invoke with the response for each region
-     * @param <R>
-     *          the response type for the coprocessor Service method
-     * @throws ServiceException
-     * @throws Throwable
-     */
-    <R extends Message> void batchCoprocessorService(Descriptors.MethodDescriptor methodDescriptor,
-      Message request, byte[] startKey, byte[] endKey, R responsePrototype,
-      Batch.Callback<R> callback) throws ServiceException, Throwable;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected value.
-     * If it does, it performs the row mutations.  If the passed value is null, the check
-     * is for the lack of column (ie: non-existence)
-     *
-     * The expected value argument of this call is on the left and the current
-     * value of the cell is on the right side of the comparison operator.
-     *
-     * Ie. eg. GREATER operator means expected value > existing <=> perform row mutations.
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param compareOp the comparison operator
-     * @param value the expected value
-     * @param mutation  mutations to perform if check succeeds
-     * @throws IOException e
-     * @return true if the new put was executed, false otherwise
-     * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-     * {@link #checkAndMutate(byte[], byte[], byte[], CompareOperator, byte[], RowMutations)}
-     */
-    @Deprecated
-    boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-        CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException;
-
-    /**
-     * Atomically checks if a row/family/qualifier value matches the expected value.
-     * If it does, it performs the row mutations.  If the passed value is null, the check
-     * is for the lack of column (ie: non-existence)
-     *
-     * The expected value argument of this call is on the left and the current
-     * value of the cell is on the right side of the comparison operator.
-     *
-     * Ie. eg. GREATER operator means expected value > existing <=> perform row mutations.
-     *
-     * @param row to check
-     * @param family column family to check
-     * @param qualifier column qualifier to check
-     * @param op the comparison operator
-     * @param value the expected value
-     * @param mutation  mutations to perform if check succeeds
-     * @throws IOException e
-     * @return true if the new put was executed, false otherwise
-     */
-    boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
-                           byte[] value, RowMutations mutation) throws IOException;
-
-    /**
-     * Get timeout of each rpc request in this Table instance. It will be overridden by a more
-     * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
-     * @see #getReadRpcTimeout(TimeUnit)
-     * @see #getWriteRpcTimeout(TimeUnit)
-     * @param unit the unit of time the timeout to be represented in
-     * @return rpc timeout in the specified time unit
-     */
-    long getRpcTimeout(TimeUnit unit);
-
-    /**
-     * Get timeout (millisecond) of each rpc request in this Table instance.
-     *
-     * @return Currently configured read timeout
-     * @deprecated use {@link #getReadRpcTimeout(TimeUnit)} or
-     *             {@link #getWriteRpcTimeout(TimeUnit)} instead
-     */
-    @Deprecated
-    int getRpcTimeout();
-
-    /**
-     * Set timeout (millisecond) of each rpc request in operations of this Table instance, will
-     * override the value of hbase.rpc.timeout in configuration.
-     * If a rpc request waiting too long, it will stop waiting and send a new request to retry until
-     * retries exhausted or operation timeout reached.
-     * <p>
-     * NOTE: This will set both the read and write timeout settings to the provided value.
-     *
-     * @param rpcTimeout the timeout of each rpc request in millisecond.
-     *
-     * @deprecated Use setReadRpcTimeout or setWriteRpcTimeout instead
-     */
-    @Deprecated
-    void setRpcTimeout(int rpcTimeout);
-
-    /**
-     * Get timeout of each rpc read request in this Table instance.
-     * @param unit the unit of time the timeout to be represented in
-     * @return read rpc timeout in the specified time unit
-     */
-    long getReadRpcTimeout(TimeUnit unit);
-
-    /**
-     * Get timeout (millisecond) of each rpc read request in this Table instance.
-     * @deprecated since 2.0 and will be removed in 3.0 version
-     *             use {@link #getReadRpcTimeout(TimeUnit)} instead
-     */
-    @Deprecated
-    int getReadRpcTimeout();
-
-    /**
-     * Set timeout (millisecond) of each rpc read request in operations of this Table instance, will
-     * override the value of hbase.rpc.read.timeout in configuration.
-     * If a rpc read request waiting too long, it will stop waiting and send a new request to retry
-     * until retries exhausted or operation timeout reached.
-     *
-     * @param readRpcTimeout the timeout for read rpc request in milliseconds
-     * @deprecated since 2.0.0, use {@link TableBuilder#setReadRpcTimeout} instead
-     */
-    @Deprecated
-    void setReadRpcTimeout(int readRpcTimeout);
-
-    /**
-     * Get timeout of each rpc write request in this Table instance.
-     * @param unit the unit of time the timeout to be represented in
-     * @return write rpc timeout in the specified time unit
-     */
-    long getWriteRpcTimeout(TimeUnit unit);
-
-    /**
-     * Get timeout (millisecond) of each rpc write request in this Table instance.
-     * @deprecated since 2.0 and will be removed in 3.0 version
-     *             use {@link #getWriteRpcTimeout(TimeUnit)} instead
-     */
-    @Deprecated
-    int getWriteRpcTimeout();
-
-    /**
-     * Set timeout (millisecond) of each rpc write request in operations of this Table instance, will
-     * override the value of hbase.rpc.write.timeout in configuration.
-     * If a rpc write request waiting too long, it will stop waiting and send a new request to retry
-     * until retries exhausted or operation timeout reached.
-     *
-     * @param writeRpcTimeout the timeout for write rpc request in milliseconds
-     * @deprecated since 2.0.0, use {@link TableBuilder#setWriteRpcTimeout} instead
-     */
-    @Deprecated
-    void setWriteRpcTimeout(int writeRpcTimeout);
-
-    /**
-     * Get timeout of each operation in Table instance.
-     * @param unit the unit of time the timeout to be represented in
-     * @return operation rpc timeout in the specified time unit
-     */
-    long getOperationTimeout(TimeUnit unit);
-
-    /**
-     * Get timeout (millisecond) of each operation for in Table instance.
-     * @deprecated since 2.0 and will be removed in 3.0 version
-     *             use {@link #getOperationTimeout(TimeUnit)} instead
-     */
-    @Deprecated
-    int getOperationTimeout();
-
-    /**
-     * Set timeout (millisecond) of each operation in this Table instance, will override the value
-     * of hbase.client.operation.timeout in configuration.
-     * Operation timeout is a top-level restriction that makes sure a blocking method will not be
-     * blocked more than this. In each operation, if rpc request fails because of timeout or
-     * other reason, it will retry until success or throw a RetriesExhaustedException. But if the
-     * total time being blocking reach the operation timeout before retries exhausted, it will break
-     * early and throw SocketTimeoutException.
-     * @param operationTimeout the total timeout of each operation in millisecond.
-     * @deprecated since 2.0.0, use {@link TableBuilder#setOperationTimeout} instead
-     */
-    @Deprecated
-    void setOperationTimeout(int operationTimeout);
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionContext.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionContext.java
index 8d19ae5..add403a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionContext.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionContext.java
@@ -19,21 +19,21 @@ package org.apache.phoenix.transaction;
 
 import java.io.IOException;
 import java.sql.SQLException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.transaction.TephraTransactionProvider.TephraTransactionClient;
 import org.apache.tephra.Transaction;
 import org.apache.tephra.Transaction.VisibilityLevel;
 import org.apache.tephra.TransactionAware;
@@ -41,48 +41,24 @@ import org.apache.tephra.TransactionCodec;
 import org.apache.tephra.TransactionConflictException;
 import org.apache.tephra.TransactionContext;
 import org.apache.tephra.TransactionFailureException;
-import org.apache.tephra.TransactionManager;
 import org.apache.tephra.TransactionSystemClient;
 import org.apache.tephra.TxConstants;
-import org.apache.tephra.distributed.PooledClientProvider;
-import org.apache.tephra.distributed.TransactionService;
-import org.apache.tephra.distributed.TransactionServiceClient;
-import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
-import org.apache.tephra.inmemory.InMemoryTxSystemClient;
-import org.apache.tephra.metrics.TxMetricsCollector;
-import org.apache.tephra.persist.HDFSTransactionStateStorage;
-import org.apache.tephra.snapshot.SnapshotCodecProvider;
-import org.apache.tephra.util.TxUtils;
 import org.apache.tephra.visibility.FenceWait;
 import org.apache.tephra.visibility.VisibilityFence;
-import org.apache.tephra.zookeeper.TephraZKClientService;
-import org.apache.twill.discovery.DiscoveryService;
-import org.apache.twill.discovery.ZKDiscoveryService;
-import org.apache.twill.internal.utils.Networks;
-import org.apache.twill.zookeeper.RetryStrategies;
-import org.apache.twill.zookeeper.ZKClientService;
-import org.apache.twill.zookeeper.ZKClientServices;
-import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Lists;
-import com.google.inject.util.Providers;
 
-public class TephraTransactionContext implements PhoenixTransactionContext {
 
+public class TephraTransactionContext implements PhoenixTransactionContext {
+    private static final Logger logger = LoggerFactory.getLogger(TephraTransactionContext.class);
     private static final TransactionCodec CODEC = new TransactionCodec();
 
-    private static TransactionSystemClient txClient = null;
-    private static ZKClientService zkClient = null;
-    private static TransactionService txService = null;
-    private static TransactionManager txManager = null;
-
     private final List<TransactionAware> txAwares;
     private final TransactionContext txContext;
     private Transaction tx;
     private TransactionSystemClient txServiceClient;
-    private TransactionFailureException e;
 
     public TephraTransactionContext() {
         this.txServiceClient = null;
@@ -92,21 +68,21 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
 
     public TephraTransactionContext(byte[] txnBytes) throws IOException {
         this();
-        this.tx = (txnBytes != null && txnBytes.length > 0) ? CODEC
-                .decode(txnBytes) : null;
+        this.tx = CODEC.decode(txnBytes);
     }
 
     public TephraTransactionContext(PhoenixConnection connection) {
-        this.txServiceClient = txClient;  
+        PhoenixTransactionClient client = connection.getQueryServices().initTransactionClient(getProvider());  
+        assert (client instanceof TephraTransactionClient);
+        this.txServiceClient = ((TephraTransactionClient)client).getTransactionClient();
         this.txAwares = Collections.emptyList();
         this.txContext = new TransactionContext(txServiceClient);
     }
 
-    public TephraTransactionContext(PhoenixTransactionContext ctx,
-            PhoenixConnection connection, boolean subTask) {
-        this.txServiceClient = txClient;  
+    private TephraTransactionContext(PhoenixTransactionContext ctx, boolean subTask) {
         assert (ctx instanceof TephraTransactionContext);
         TephraTransactionContext tephraTransactionContext = (TephraTransactionContext) ctx;
+        this.txServiceClient = tephraTransactionContext.txServiceClient;  
 
         if (subTask) {
             this.tx = tephraTransactionContext.getTransaction();
@@ -116,42 +92,13 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
             this.txAwares = Collections.emptyList();
             this.txContext = tephraTransactionContext.getContext();
         }
-
-        this.e = null;
-    }
-
-    @Override
-    public void setInMemoryTransactionClient(Configuration config) {
-        TransactionManager txnManager = new TransactionManager(config);
-        txClient = this.txServiceClient = new InMemoryTxSystemClient(txnManager);
     }
 
     @Override
-    public ZKClientService setTransactionClient(Configuration config, ReadOnlyProps props, ConnectionInfo connectionInfo) {
-        String zkQuorumServersString = props.get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
-        if (zkQuorumServersString==null) {
-            zkQuorumServersString = connectionInfo.getZookeeperQuorum()+":"+connectionInfo.getPort();
-        }
-
-        int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client
-        ZKClientService txZKClientService  = ZKClientServices.delegate(
-            ZKClients.reWatchOnExpire(
-                ZKClients.retryOnFailure(
-                     new TephraZKClientService(zkQuorumServersString, timeOut, null,
-                             ArrayListMultimap.<String, byte[]>create()), 
-                         RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                     )
-                );
-        txZKClientService.startAndWait();
-        ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(txZKClientService);
-        PooledClientProvider pooledClientProvider = new PooledClientProvider(
-                config, zkDiscoveryService);
-        txClient = this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
-        
-        return txZKClientService;
+    public TransactionFactory.Provider getProvider() {
+        return TransactionFactory.Provider.TEPHRA;
     }
-
+    
     @Override
     public void begin() throws SQLException {
         if (txContext == null) {
@@ -180,8 +127,6 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
         try {
             txContext.finish();
         } catch (TransactionFailureException e) {
-            this.e = e;
-
             if (e instanceof TransactionConflictException) {
                 throw new SQLExceptionInfo.Builder(
                         SQLExceptionCode.TRANSACTION_CONFLICT_EXCEPTION)
@@ -203,14 +148,8 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
         }
 
         try {
-            if (e != null) {
-                txContext.abort(e);
-                e = null;
-            } else {
-                txContext.abort();
-            }
+            txContext.abort();
         } catch (TransactionFailureException e) {
-            this.e = null;
             throw new SQLExceptionInfo.Builder(
                     SQLExceptionCode.TRANSACTION_FAILED)
                     .setMessage(e.getMessage()).setRootCause(e).build()
@@ -248,7 +187,7 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
     }
 
     @Override
-    public void commitDDLFence(PTable dataTable, Logger logger)
+    public void commitDDLFence(PTable dataTable)
             throws SQLException {
         byte[] key = dataTable.getName().getBytes();
 
@@ -275,7 +214,12 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
         }
     }
 
+    @Override
     public void markDMLFence(PTable table) {
+        if (table.getType() == PTableType.INDEX) {
+            return;
+        }
+        
         byte[] logicalKey = table.getName().getBytes();
         TransactionAware logicalTxAware = VisibilityFence.create(logicalKey);
 
@@ -299,6 +243,9 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
 
     @Override
     public void join(PhoenixTransactionContext ctx) {
+        if (ctx == PhoenixTransactionContext.NULL_CONTEXT) {
+            return;
+        }
         assert (ctx instanceof TephraTransactionContext);
         TephraTransactionContext tephraContext = (TephraTransactionContext) ctx;
 
@@ -324,7 +271,6 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
     public void reset() {
         tx = null;
         txAwares.clear();
-        this.e = null;
     }
 
     @Override
@@ -405,86 +351,15 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
         assert (tx != null);
 
         try {
-            return CODEC.encode(tx);
+            byte[] encodedTxBytes = CODEC.encode(tx);
+            encodedTxBytes = Arrays.copyOf(encodedTxBytes, encodedTxBytes.length + 1);
+            encodedTxBytes[encodedTxBytes.length - 1] = getProvider().getCode();
+            return encodedTxBytes;
         } catch (IOException e) {
             throw new SQLException(e);
         }
     }
 
-    @Override
-    public long getMaxTransactionsPerSecond() {
-        return TxConstants.MAX_TX_PER_MS;
-    }
-
-    @Override
-    public boolean isPreExistingVersion(long version) {
-        return TxUtils.isPreExistingVersion(version);
-    }
-
-    @Override
-    public RegionObserver getCoprocessor() {
-        return new TransactionProcessor();
-    }
-
-    @Override
-    public byte[] getFamilyDeleteMarker() {
-        return TxConstants.FAMILY_DELETE_QUALIFIER;
-    }
-
-    @Override
-    public void setTxnConfigs(Configuration config, String tmpFolder, int defaultTxnTimeoutSeconds) throws IOException {
-        config.setBoolean(TxConstants.Manager.CFG_DO_PERSIST, false);
-        config.set(TxConstants.Service.CFG_DATA_TX_CLIENT_RETRY_STRATEGY, "n-times");
-        config.setInt(TxConstants.Service.CFG_DATA_TX_CLIENT_ATTEMPTS, 1);
-        config.setInt(TxConstants.Service.CFG_DATA_TX_BIND_PORT, Networks.getRandomPort());
-        config.set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, tmpFolder);
-        config.setInt(TxConstants.Manager.CFG_TX_TIMEOUT, defaultTxnTimeoutSeconds);
-        config.unset(TxConstants.Manager.CFG_TX_HDFS_USER);
-        config.setLong(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5L);
-    }
-
-    @Override
-    public void setupTxManager(Configuration config, String url) throws SQLException {
-
-        if (txService != null) {
-            return;
-        }
-
-        ConnectionInfo connInfo = ConnectionInfo.create(url);
-        zkClient = ZKClientServices.delegate(
-          ZKClients.reWatchOnExpire(
-            ZKClients.retryOnFailure(
-              ZKClientService.Builder.of(connInfo.getZookeeperConnectionString())
-                .setSessionTimeout(config.getInt(HConstants.ZK_SESSION_TIMEOUT,
-                        HConstants.DEFAULT_ZK_SESSION_TIMEOUT))
-                .build(),
-              RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS)
-            )
-          )
-        );
-        zkClient.startAndWait();
-
-        DiscoveryService discovery = new ZKDiscoveryService(zkClient);
-        txManager = new TransactionManager(config, new HDFSTransactionStateStorage(config, new SnapshotCodecProvider(config), new TxMetricsCollector()), new TxMetricsCollector());
-        txService = new TransactionService(config, zkClient, discovery, Providers.of(txManager));
-        txService.startAndWait();
-    }
-
-    @Override
-    public void tearDownTxManager() {
-        try {
-            if (txService != null) txService.stopAndWait();
-        } finally {
-            try {
-                if (zkClient != null) zkClient.stopAndWait();
-            } finally {
-                txService = null;
-                zkClient = null;
-                txManager = null;
-            }
-        }
-    }
-
     /**
      * TephraTransactionContext specific functions
      */
@@ -510,4 +385,17 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
             txAware.startTx(tx);
         }
     }
+
+    @Override
+    public PhoenixTransactionContext newTransactionContext(PhoenixTransactionContext context, boolean subTask) {
+        return new TephraTransactionContext(context, subTask);
+    }
+    
+    @Override
+    public Table getTransactionalTable(Table htable, boolean isImmutable) {
+        TransactionAwareHTable transactionAwareHTable = new TransactionAwareHTable(htable, isImmutable ? TxConstants.ConflictDetection.NONE : TxConstants.ConflictDetection.ROW);
+        this.addTransactionAware(transactionAwareHTable);
+        return transactionAwareHTable;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
index 075c922..2e52efa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionProvider.java
@@ -18,16 +18,37 @@
 package org.apache.phoenix.transaction;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
+import org.apache.tephra.TransactionManager;
+import org.apache.tephra.TransactionSystemClient;
 import org.apache.tephra.TxConstants;
+import org.apache.tephra.distributed.PooledClientProvider;
+import org.apache.tephra.distributed.TransactionService;
+import org.apache.tephra.distributed.TransactionServiceClient;
+import org.apache.tephra.inmemory.InMemoryTxSystemClient;
+import org.apache.tephra.metrics.TxMetricsCollector;
+import org.apache.tephra.persist.HDFSTransactionStateStorage;
+import org.apache.tephra.snapshot.SnapshotCodecProvider;
+import org.apache.tephra.zookeeper.TephraZKClientService;
+import org.apache.twill.discovery.DiscoveryService;
+import org.apache.twill.discovery.ZKDiscoveryService;
+import org.apache.twill.zookeeper.RetryStrategies;
+import org.apache.twill.zookeeper.ZKClientService;
+import org.apache.twill.zookeeper.ZKClientServices;
+import org.apache.twill.zookeeper.ZKClients;
 
-public class TephraTransactionProvider implements TransactionProvider {
+import com.google.common.collect.ArrayListMultimap;
+import com.google.inject.util.Providers;
+
+public class TephraTransactionProvider implements PhoenixTransactionProvider {
     private static final TephraTransactionProvider INSTANCE = new TephraTransactionProvider();
     
     public static final TephraTransactionProvider getInstance() {
@@ -37,12 +58,6 @@ public class TephraTransactionProvider implements TransactionProvider {
     private TephraTransactionProvider() {
     }
     
-    
-    @Override
-    public PhoenixTransactionContext getTransactionContext()  {
-        return new TephraTransactionContext();
-    }
-
     @Override
     public PhoenixTransactionContext getTransactionContext(byte[] txnBytes) throws IOException {
        return new TephraTransactionContext(txnBytes);
@@ -54,23 +69,129 @@ public class TephraTransactionProvider implements TransactionProvider {
     }
 
     @Override
-    public PhoenixTransactionContext getTransactionContext(PhoenixTransactionContext contex, PhoenixConnection connection, boolean subTask) {
-        return new TephraTransactionContext(contex, connection, subTask);
+    public PhoenixTransactionClient getTransactionClient(Configuration config, ConnectionInfo connectionInfo) {
+        if (connectionInfo.isConnectionless()) {
+            TransactionManager txnManager = new TransactionManager(config);
+            TransactionSystemClient txClient = new InMemoryTxSystemClient(txnManager);
+            return new TephraTransactionClient(txClient);
+            
+        }
+        String zkQuorumServersString = config.get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
+        if (zkQuorumServersString==null) {
+            zkQuorumServersString = connectionInfo.getZookeeperConnectionString();
+        }
+
+        int timeOut = config.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
+        // Create instance of the tephra zookeeper client
+        ZKClientService zkClientService  = ZKClientServices.delegate(
+            ZKClients.reWatchOnExpire(
+                ZKClients.retryOnFailure(
+                     new TephraZKClientService(zkQuorumServersString, timeOut, null,
+                             ArrayListMultimap.<String, byte[]>create()), 
+                         RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                     )
+                );
+        //txZKClientService.startAndWait();
+        ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
+        PooledClientProvider pooledClientProvider = new PooledClientProvider(
+                config, zkDiscoveryService);
+        TransactionServiceClient txClient = new TransactionServiceClient(config,pooledClientProvider);
+        TephraTransactionClient client = new TephraTransactionClient(zkClientService, txClient);
+        client.start();
+        
+        return client;
     }
 
     @Override
-    public PhoenixTransactionalTable getTransactionalTable(PhoenixTransactionContext ctx, Table htable) {
-        return new TephraTransactionTable(ctx, htable);
+    public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connInfo) {
+        ZKClientService zkClient = ZKClientServices.delegate(
+          ZKClients.reWatchOnExpire(
+            ZKClients.retryOnFailure(
+              ZKClientService.Builder.of(connInfo.getZookeeperConnectionString())
+                .setSessionTimeout(config.getInt(HConstants.ZK_SESSION_TIMEOUT,
+                        HConstants.DEFAULT_ZK_SESSION_TIMEOUT))
+                .build(),
+              RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS)
+            )
+          )
+        );
+
+        //zkClient.startAndWait();
+        DiscoveryService discovery = new ZKDiscoveryService(zkClient);
+        TransactionManager txManager = new TransactionManager(config, new HDFSTransactionStateStorage(config, new SnapshotCodecProvider(config), new TxMetricsCollector()), new TxMetricsCollector());
+        TransactionService txService = new TransactionService(config, zkClient, discovery, Providers.of(txManager));
+        TephraTransactionService service = new TephraTransactionService(zkClient, txService);
+        //txService.startAndWait();            
+        service.start();
+        return service;
+    }
+
+    static class TephraTransactionService implements PhoenixTransactionService {
+        private final ZKClientService zkClient;
+        private final TransactionService txService;
+
+        public TephraTransactionService(ZKClientService zkClient, TransactionService txService) {
+            this.zkClient = zkClient;
+            this.txService = txService;
+        }
+        
+        public void start() {
+            zkClient.startAndWait();
+            txService.startAndWait();            
+        }
+        
+        @Override
+        public void close() throws IOException {
+            try {
+                if (txService != null) txService.stopAndWait();
+            } finally {
+                if (zkClient != null) zkClient.stopAndWait();
+            }
+        }
+        
     }
     
-    @Override
-    public Cell newDeleteFamilyMarker(byte[] row, byte[] family, long timestamp) {
-        return CellUtil.createCell(row, family, TxConstants.FAMILY_DELETE_QUALIFIER, timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
+    static class TephraTransactionClient implements PhoenixTransactionClient {
+        private final ZKClientService zkClient;
+        private final TransactionSystemClient txClient;
+
+        public TephraTransactionClient(TransactionSystemClient txClient) {
+            this(null, txClient);
+        }
+        
+        public TephraTransactionClient(ZKClientService zkClient, TransactionSystemClient txClient) {
+            this.zkClient = zkClient;
+            this.txClient = txClient;
+        }
+        
+        public void start() {
+            zkClient.startAndWait();
+        }
+        
+        public TransactionSystemClient getTransactionClient() {
+            return txClient;
+        }
+        
+        @Override
+        public void close() throws IOException {
+            zkClient.stopAndWait();
+        }
+        
     }
     
     @Override
-    public Cell newDeleteColumnMarker(byte[] row, byte[] family, byte[] qualifier, long timestamp) {
-        return CellUtil.createCell(row, family, qualifier, timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
+    public Class<? extends RegionObserver> getCoprocessor() {
+        return TephraTransactionalProcessor.class;
+    }
+
+    @Override
+    public Provider getProvider() {
+        return TransactionFactory.Provider.TEPHRA;
+    }
+
+    @Override
+    public boolean isUnsupported(Feature feature) {
+        return false;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index e28e98b..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,360 +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.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompareOperator;
-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.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.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-import org.apache.tephra.TxConstants;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-    private TransactionAwareHTable transactionAwareHTable;
-
-    private TephraTransactionContext tephraTransactionContext;
-
-    public TephraTransactionTable(PhoenixTransactionContext ctx, Table hTable) {
-        this(ctx, hTable, null);
-    }
-
-    public TephraTransactionTable(PhoenixTransactionContext ctx, Table hTable, PTable pTable) {
-
-        assert(ctx instanceof TephraTransactionContext);
-
-        tephraTransactionContext = (TephraTransactionContext) ctx;
-
-        transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != null && pTable.isImmutableRows())
-                ? TxConstants.ConflictDetection.NONE : TxConstants.ConflictDetection.ROW);
-
-        tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-        if (pTable != null && pTable.getType() != PTableType.INDEX) {
-            tephraTransactionContext.markDMLFence(pTable);
-        }
-    }
-
-    @Override
-    public Result get(Get get) throws IOException {
-        return transactionAwareHTable.get(get);
-    }
-
-    @Override
-    public void put(Put put) throws IOException {
-        transactionAwareHTable.put(put);
-    }
-
-    @Override
-    public void delete(Delete delete) throws IOException {
-        transactionAwareHTable.delete(delete);
-    }
-
-    @Override
-    public ResultScanner getScanner(Scan scan) throws IOException {
-        return transactionAwareHTable.getScanner(scan);
-    }
-
-
-    @Override
-    public Configuration getConfiguration() {
-        return transactionAwareHTable.getConfiguration();
-    }
-
-    @Override
-    public HTableDescriptor getTableDescriptor() throws IOException {
-        return transactionAwareHTable.getTableDescriptor();
-    }
-
-    @Override
-    public boolean exists(Get get) throws IOException {
-        return transactionAwareHTable.exists(get);
-    }
-
-    @Override
-    public Result[] get(List<Get> gets) throws IOException {
-        return transactionAwareHTable.get(gets);
-    }
-
-    @Override
-    public ResultScanner getScanner(byte[] family) throws IOException {
-        return transactionAwareHTable.getScanner(family);
-    }
-
-    @Override
-    public ResultScanner getScanner(byte[] family, byte[] qualifier)
-            throws IOException {
-        return transactionAwareHTable.getScanner(family, qualifier);
-    }
-
-    @Override
-    public void put(List<Put> puts) throws IOException {
-        transactionAwareHTable.put(puts);
-    }
-
-    @Override
-    public void delete(List<Delete> deletes) throws IOException {
-        transactionAwareHTable.delete(deletes);
-    }
-
-    @Override
-    public void close() throws IOException {
-        transactionAwareHTable.close();
-    }
-
-
-    @Override
-    public TableName getName() {
-        return transactionAwareHTable.getName();
-    }
-
-    @Override
-    public boolean[] existsAll(List<Get> gets) throws IOException {
-        return transactionAwareHTable.existsAll(gets);
-    }
-
-    @Override
-    public void batch(List<? extends Row> actions, Object[] results)
-            throws IOException, InterruptedException {
-        transactionAwareHTable.batch(actions, results);
-    }
-
-    @Override
-    public <R> void batchCallback(List<? extends Row> actions,
-            Object[] results, Callback<R> callback) throws IOException,
-            InterruptedException {
-        transactionAwareHTable.batchCallback(actions, results, callback);
-    }
-
-    @Override
-    public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-            byte[] value, Put put) throws IOException {
-        return transactionAwareHTable.checkAndPut(row, family, qualifier, value, put);
-    }
-
-    @Override
-    public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-            CompareOp compareOp, byte[] value, Put put) throws IOException {
-        return transactionAwareHTable.checkAndPut(row, family, qualifier, compareOp, value, put);
-    }
-
-    @Override
-    public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-            byte[] value, Delete delete) throws IOException {
-        return transactionAwareHTable.checkAndDelete(row, family, qualifier, value, delete);
-    }
-
-    @Override
-    public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-            CompareOp compareOp, byte[] value, Delete delete)
-            throws IOException {
-        return transactionAwareHTable.checkAndDelete(row, family, qualifier, compareOp, value, delete);
-    }
-
-    @Override
-    public void mutateRow(RowMutations rm) throws IOException {
-        transactionAwareHTable.mutateRow(rm);
-    }
-
-    @Override
-    public Result append(Append append) throws IOException {
-        return transactionAwareHTable.append(append);
-    }
-
-    @Override
-    public Result increment(Increment increment) throws IOException {
-        return transactionAwareHTable.increment(increment);
-    }
-
-    @Override
-    public long incrementColumnValue(byte[] row, byte[] family,
-            byte[] qualifier, long amount) throws IOException {
-        return transactionAwareHTable.incrementColumnValue(row, family, qualifier, amount);
-    }
-
-    @Override
-    public long incrementColumnValue(byte[] row, byte[] family,
-            byte[] qualifier, long amount, Durability durability)
-            throws IOException {
-        return transactionAwareHTable.incrementColumnValue(row, family, qualifier, amount, durability);
-    }
-
-    @Override
-    public CoprocessorRpcChannel coprocessorService(byte[] row) {
-        return transactionAwareHTable.coprocessorService(row);
-    }
-
-    @Override
-    public <T extends Service, R> Map<byte[], R> coprocessorService(
-            Class<T> service, byte[] startKey, byte[] endKey,
-            Call<T, R> callable) throws ServiceException, Throwable {
-        return transactionAwareHTable.coprocessorService(service, startKey, endKey, callable);
-    }
-
-    @Override
-    public <T extends Service, R> void coprocessorService(Class<T> service,
-            byte[] startKey, byte[] endKey, Call<T, R> callable,
-            Callback<R> callback) throws ServiceException, Throwable {
-        transactionAwareHTable.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 transactionAwareHTable.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 {
-        transactionAwareHTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype, callback);
-    }
-
-    @Override
-    public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-            CompareOp compareOp, byte[] value, RowMutations mutation)
-            throws IOException {
-        return transactionAwareHTable.checkAndMutate(row, family, qualifier, compareOp, value, mutation);
-    }
-
-    @Override
-    public void setOperationTimeout(int operationTimeout) {
-        transactionAwareHTable.setOperationTimeout(operationTimeout);
-    }
-
-    @Override
-    public int getOperationTimeout() {
-        return transactionAwareHTable.getOperationTimeout();
-    }
-
-    @Override
-    public void setRpcTimeout(int rpcTimeout) {
-        transactionAwareHTable.setRpcTimeout(rpcTimeout);
-    }
-
-    @Override
-    public int getRpcTimeout() {
-        return transactionAwareHTable.getRpcTimeout();
-    }
-
-    @Override
-    public TableDescriptor getDescriptor() throws IOException {
-        return transactionAwareHTable.getDescriptor();
-    }
-
-    @Override
-    public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
-            byte[] value, Put put) throws IOException {
-        return transactionAwareHTable.checkAndPut(row, family, qualifier, op, value, put);
-    }
-
-    @Override
-    public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
-            byte[] value, Delete delete) throws IOException {
-        return transactionAwareHTable.checkAndDelete(row, family, qualifier, op, value, delete);
-    }
-
-    @Override
-    public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
-        return transactionAwareHTable.checkAndMutate(row, family);
-    }
-
-    @Override
-    public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
-            byte[] value, RowMutations mutation) throws IOException {
-        return transactionAwareHTable.checkAndMutate(row, family, qualifier, op, value, mutation);
-    }
-
-    @Override
-    public int getReadRpcTimeout() {
-        return transactionAwareHTable.getReadRpcTimeout();
-    }
-
-    @Override
-    public void setReadRpcTimeout(int readRpcTimeout) {
-        transactionAwareHTable.setReadRpcTimeout(readRpcTimeout);
-    }
-
-    @Override
-    public int getWriteRpcTimeout() {
-        return transactionAwareHTable.getWriteRpcTimeout();
-    }
-
-    @Override
-    public void setWriteRpcTimeout(int writeRpcTimeout) {
-        transactionAwareHTable.setWriteRpcTimeout(writeRpcTimeout);
-    }
-
-    @Override
-    public boolean[] exists(List<Get> gets) throws IOException {
-        return transactionAwareHTable.exists(gets);
-    }
-
-    @Override
-    public long getRpcTimeout(TimeUnit unit) {
-        return transactionAwareHTable.getRpcTimeout();
-    }
-
-    @Override
-    public long getReadRpcTimeout(TimeUnit unit) {
-        return transactionAwareHTable.getReadRpcTimeout(unit);
-    }
-
-    @Override
-    public long getWriteRpcTimeout(TimeUnit unit) {
-        return transactionAwareHTable.getWriteRpcTimeout(unit);
-    }
-
-    @Override
-    public long getOperationTimeout(TimeUnit unit) {
-        return transactionAwareHTable.getOperationTimeout(unit);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
index f32764b..62bd808 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
@@ -17,24 +17,55 @@
  */
 package org.apache.phoenix.transaction;
 
+import java.io.IOException;
+
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+
+
 public class TransactionFactory {
-    enum TransactionProcessor {
-        Tephra,
-        Omid
+    public enum Provider {
+        TEPHRA((byte)1, TephraTransactionProvider.getInstance()),
+        OMID((byte)2, OmidTransactionProvider.getInstance());
+        
+        private final byte code;
+        private final PhoenixTransactionProvider provider;
+        
+        Provider(byte code, PhoenixTransactionProvider provider) {
+            this.code = code;
+            this.provider = provider;
+        }
+        
+        public byte getCode() {
+            return this.code;
+        }
+
+        public static Provider fromCode(int code) {
+            if (code < 1 || code > Provider.values().length) {
+                throw new IllegalArgumentException("Invalid TransactionFactory.Provider " + code);
+            }
+            return Provider.values()[code-1];
+        }
+        
+        public static Provider getDefault() {
+            return TEPHRA;
+        }
+
+        public PhoenixTransactionProvider getTransactionProvider()  {
+            return provider;
+        }
     }
 
-    static public TransactionProvider getTransactionProvider() {
-        return TephraTransactionProvider.getInstance();
+    public static PhoenixTransactionProvider getTransactionProvider(Provider provider) {
+        return provider.getTransactionProvider();
     }
     
-    static public TransactionProvider getTransactionProvider(TransactionProcessor processor) {
-        switch (processor) {
-        case Tephra:
-            return TephraTransactionProvider.getInstance();
-        case Omid:
-            return OmidTransactionProvider.getInstance();
-        default:
-            throw new IllegalArgumentException("Unknown transaction processor: " + processor);
+    public static PhoenixTransactionContext getTransactionContext(byte[] txState, int clientVersion) throws IOException {
+        if (txState == null || txState.length == 0) {
+            return null;
         }
+        Provider provider = (clientVersion < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0) 
+                ? Provider.OMID
+                : Provider.fromCode(txState[txState.length-1]);
+        return provider.getTransactionProvider().getTransactionContext(txState);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionProvider.java
deleted file mode 100644
index 84211ab..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionProvider.java
+++ /dev/null
@@ -1,36 +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.phoenix.transaction;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-
-public interface TransactionProvider {
-    public PhoenixTransactionContext getTransactionContext();
-    public PhoenixTransactionContext getTransactionContext(byte[] txnBytes) throws IOException;
-    public PhoenixTransactionContext getTransactionContext(PhoenixConnection connection);
-    public PhoenixTransactionContext getTransactionContext(PhoenixTransactionContext contex, PhoenixConnection connection, boolean subTask);
-    
-    public PhoenixTransactionalTable getTransactionalTable(PhoenixTransactionContext ctx, Table htable);
-    
-    public Cell newDeleteFamilyMarker(byte[] row, byte[] family, long timestamp);
-    public Cell newDeleteColumnMarker(byte[] row, byte[] family, byte[] qualifier, long timestamp);
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index a0912f3..e85732a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -87,7 +87,6 @@ import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.ValueBitSet;
 import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.transaction.TransactionFactory;
 
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
@@ -1514,7 +1513,7 @@ public class PhoenixRuntime {
      * @return wall clock time in milliseconds (i.e. Epoch time) of a given Cell time stamp.
      */
     public static long getWallClockTimeFromCellTimeStamp(long tsOfCell) {
-        return TransactionFactory.getTransactionProvider().getTransactionContext().isPreExistingVersion(tsOfCell) ? tsOfCell : TransactionUtil.convertToMilliseconds(tsOfCell);
+        return TransactionUtil.isTransactionalTimestamp(tsOfCell) ? TransactionUtil.convertToMilliseconds(tsOfCell) : tsOfCell;
     }
 
     public static long getCurrentScn(ReadOnlyProps props) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index f0bc2d6..914417c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -59,6 +59,7 @@ import org.apache.phoenix.filter.DistinctPrefixFilter;
 import org.apache.phoenix.filter.MultiEncodedCQKeyValueComparisonFilter;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
@@ -88,6 +89,8 @@ import com.google.common.collect.Lists;
  */
 public class ScanUtil {
     public static final int[] SINGLE_COLUMN_SLOT_SPAN = new int[1];
+    public static final int UNKNOWN_CLIENT_VERSION = VersionUtil.encodeVersion(4, 4, 0);
+
     /*
      * Max length that we fill our key when we turn an inclusive key
      * into a exclusive key.
@@ -931,5 +934,17 @@ public class ScanUtil {
     public static boolean isIndexRebuild(Scan scan) {
         return scan.getAttribute((BaseScannerRegionObserver.REBUILD_INDEXES)) != null;
     }
+ 
+    public static int getClientVersion(Scan scan) {
+        int clientVersion = UNKNOWN_CLIENT_VERSION;
+        byte[] clientVersionBytes = scan.getAttribute(BaseScannerRegionObserver.CLIENT_VERSION);
+        if (clientVersionBytes != null) {
+            clientVersion = Bytes.toInt(clientVersionBytes);
+        }
+        return clientVersion;
+    }
     
+    public static void setClientVersion(Scan scan, int version) {
+        scan.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, Bytes.toBytes(version));
+    }
 }


[5/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

Posted by ja...@apache.org.
PHOENIX-4605 Support running multiple transaction providers


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/15fa00fa
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/15fa00fa
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/15fa00fa

Branch: refs/heads/5.x-HBase-2.0
Commit: 15fa00fa59472c5f61a211af5c723b4803a3cac3
Parents: 5d4cb80
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Apr 12 22:30:56 2018 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java        |  34 +-
 .../phoenix/tx/FlappingTransactionIT.java       |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  12 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java       |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java    |   2 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   2 +-
 .../compile/TupleProjectionCompiler.java        |   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   3 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  72 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   3 +-
 .../coprocessor/MetaDataRegionObserver.java     |   4 +-
 .../PhoenixTransactionalProcessor.java          |  52 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java           |  52 ++
 .../UngroupedAggregateRegionObserver.java       |  10 +-
 .../coprocessor/generated/PTableProtos.java     | 110 ++-
 .../phoenix/exception/SQLExceptionCode.java     |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  75 +-
 .../PhoenixTxIndexMutationGenerator.java        |  10 +-
 .../phoenix/expression/ExpressionType.java      | 119 +--
 .../TransactionProviderNameFunction.java        |  81 ++
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java        |   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java      |   7 +-
 .../NonAggregateRegionScannerFactory.java       |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   4 +
 .../query/ConnectionQueryServicesImpl.java      |  69 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/QueryConstants.java    | 108 ++-
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java     |  10 +-
 .../apache/phoenix/schema/DelegateTable.java    |   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 ++-
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  67 +-
 .../apache/phoenix/schema/TableProperty.java    |  18 +
 .../transaction/OmidTransactionContext.java     |  56 +-
 .../transaction/OmidTransactionProvider.java    |  54 +-
 .../transaction/OmidTransactionTable.java       | 380 ---------
 .../transaction/PhoenixTransactionClient.java   |  23 +
 .../transaction/PhoenixTransactionContext.java  | 165 ++--
 .../transaction/PhoenixTransactionProvider.java |  51 ++
 .../transaction/PhoenixTransactionService.java  |  24 +
 .../transaction/PhoenixTransactionalTable.java  | 798 -------------------
 .../transaction/TephraTransactionContext.java   | 198 +----
 .../transaction/TephraTransactionProvider.java  | 161 +++-
 .../transaction/TephraTransactionTable.java     | 360 ---------
 .../phoenix/transaction/TransactionFactory.java |  57 +-
 .../transaction/TransactionProvider.java        |  36 -
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java    |  49 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   4 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java    |  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto          |   1 +
 70 files changed, 1320 insertions(+), 2338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index aeb892e..7e1befe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -36,7 +36,7 @@ import java.util.Collection;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
+import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
@@ -748,7 +748,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             Table htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
-            assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+            assertFalse(htable.getTableDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
             assertFalse(phoenixConn.getTable(new PTableKey(null, baseTableName)).isTransactional());
             assertFalse(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).isTransactional());
             
@@ -757,7 +757,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             // query the view to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM " + viewOfTable);
             htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
-            assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+            assertTrue(htable.getTableDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
             assertTrue(phoenixConn.getTable(new PTableKey(null, baseTableName)).isTransactional());
             assertTrue(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).isTransactional());
         } 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
index a1ad1ad..3db93b0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
 
+import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -29,7 +30,13 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.query.ConnectionQueryServicesImpl;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.transaction.PhoenixTransactionClient;
+import org.apache.phoenix.transaction.PhoenixTransactionService;
+import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.SQLCloseables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
@@ -42,9 +49,11 @@ import com.google.common.collect.Sets;
  * @since 0.1
  */
 public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl {
+    private static final Logger logger = LoggerFactory.getLogger(ConnectionQueryServicesTestImpl.class);
     protected int NUM_SLAVES_BASE = 1; // number of slaves for the cluster
     // Track open connections to free them on close as unit tests don't always do this.
     private Set<PhoenixConnection> connections = Sets.newHashSet();
+    private final PhoenixTransactionService[] txServices = new PhoenixTransactionService[TransactionFactory.Provider.values().length];
     
     public ConnectionQueryServicesTestImpl(QueryServices services, ConnectionInfo info, Properties props) throws SQLException {
         super(services, info, props);
@@ -68,12 +77,33 @@ public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl
                 // Make copy to prevent ConcurrentModificationException (TODO: figure out why this is necessary)
                 connections = new ArrayList<>(this.connections);
                 this.connections = Sets.newHashSet();
+                
+                // shut down the tx client service if we created one to support transactions
+                for (PhoenixTransactionService service : txServices) {
+                    if (service != null) {
+                        try {
+                            service.close();
+                        } catch (IOException e) {
+                            logger.warn(e.getMessage(), e);
+                        }
+                    }
+                }
+
             }
             SQLCloseables.closeAll(connections);
-             long unfreedBytes = clearCache();
-             assertEquals("Found unfreed bytes in server-side cache", 0, unfreedBytes);
+            long unfreedBytes = clearCache();
+            assertEquals("Found unfreed bytes in server-side cache", 0, unfreedBytes);
         } finally {
             super.close();
         }
     }
+    
+    @Override
+    public synchronized PhoenixTransactionClient initTransactionClient(Provider provider) {
+        PhoenixTransactionService txService = txServices[provider.ordinal()];
+        if (txService == null) {
+            txService = txServices[provider.ordinal()] = provider.getTransactionProvider().getTransactionService(config, connectionInfo);
+        }
+        return super.initTransactionClient(provider);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
index 4d8e297..52d58c9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
@@ -43,7 +43,6 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
-import org.apache.phoenix.transaction.PhoenixTransactionalTable;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.TestUtil;
@@ -227,9 +226,9 @@ public class FlappingTransactionIT extends ParallelStatsDisabledIT {
         }
 
         PhoenixTransactionContext txContext =
-              TransactionFactory.getTransactionProvider().getTransactionContext(pconn);
-        PhoenixTransactionalTable txTable =
-              TransactionFactory.getTransactionProvider().getTransactionalTable(txContext, htable);
+              TransactionFactory.getTransactionProvider(TransactionFactory.Provider.TEPHRA).getTransactionContext(pconn);
+        Table txTable =
+                txContext.getTransactionalTable(htable, false);
 
         txContext.begin();
 
@@ -279,9 +278,9 @@ public class FlappingTransactionIT extends ParallelStatsDisabledIT {
         
         // Repeat the same as above, but this time abort the transaction
         txContext =
-              TransactionFactory.getTransactionProvider().getTransactionContext(pconn);
+              TransactionFactory.getTransactionProvider(TransactionFactory.Provider.TEPHRA).getTransactionContext(pconn);
         txTable =
-              TransactionFactory.getTransactionProvider().getTransactionalTable(txContext, htable);
+              txContext.getTransactionalTable(htable, false);
 
         txContext.begin();
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
index b25ec03..1f02386 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
+import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -282,9 +282,9 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         conn.createStatement().execute("ALTER TABLE " + nonTxTableName + " SET TRANSACTIONAL=true");
         
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
-        assertTrue(htable.getDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(index));
-        assertTrue(htable.getDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
 
         conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (4, 'c')");
         ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ k FROM " + nonTxTableName + " WHERE v IS NULL");
@@ -359,7 +359,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         assertFalse(rs.next());
         
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
-        assertFalse(htable.getDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertFalse(htable.getDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
         assertEquals(1,conn.unwrap(PhoenixConnection.class).getQueryServices().
                 getTableDescriptor(Bytes.toBytes("SYSTEM." + nonTxTableName)).
                 getColumnFamily(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES).getMaxVersions());
@@ -377,7 +377,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         PTable table = pconn.getTable(new PTableKey(null, t1));
         Table htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
         assertTrue(table.isTransactional());
-        assertTrue(htable.getDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
         
         try {
             ddl = "ALTER TABLE " + t1 + " SET transactional=false";
@@ -411,7 +411,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         table = pconn.getTable(new PTableKey(null, t1));
         htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
         assertTrue(table.isTransactional());
-        assertTrue(htable.getDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index e743bfd..88fba47 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -21,6 +21,7 @@ import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -45,6 +46,7 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -130,6 +132,7 @@ public class TransactionIT  extends ParallelStatsDisabledIT {
             assertTrue(rs.next());
             assertEquals("Transactional table was not marked as transactional in JDBC API.",
                 "true", rs.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
+            assertEquals(TransactionFactory.Provider.TEPHRA.name(), rs.getString(PhoenixDatabaseMetaData.TRANSACTION_PROVIDER));
 
             String nonTransactTableName = generateUniqueName();
             Statement stmt2 = conn.createStatement();
@@ -140,6 +143,15 @@ public class TransactionIT  extends ParallelStatsDisabledIT {
             assertTrue(rs2.next());
             assertEquals("Non-transactional table was marked as transactional in JDBC API.",
                 "false", rs2.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
+            assertNull(rs2.getString(PhoenixDatabaseMetaData.TRANSACTION_PROVIDER));
+            
+            try {
+                stmt.execute("CREATE TABLE " + transactTableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " +
+                        "TRANSACTION_PROVIDER=foo");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.UNKNOWN_TRANSACTION_PROVIDER.getErrorCode(), e.getErrorCode());
+            }
         }
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index 5083016..2c71ee3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -38,6 +38,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.transaction.PhoenixTransactionContext.PhoenixVisibilityLevel;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Ignore;
@@ -267,7 +268,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
 		ResultSet rs;
 		MutationState state = conn.unwrap(PhoenixConnection.class)
 				.getMutationState();
-		state.startTransaction();
+		state.startTransaction(TransactionFactory.Provider.TEPHRA);
 		long wp = state.getWritePointer();
 		conn.createStatement().execute(
 				"upsert into " + fullTableName + " select max(id)+1, 'a4', 'b4' from " + fullTableName + "");
@@ -331,7 +332,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
 			conn.commit();
 
 	        MutationState state = conn.unwrap(PhoenixConnection.class).getMutationState();
-	        state.startTransaction();
+	        state.startTransaction(TransactionFactory.Provider.TEPHRA);
 	        long wp = state.getWritePointer();
 	        conn.createStatement().execute("delete from " + fullTableName + "1 where id1=fk1b AND fk1b=id1");
 	        assertEquals(PhoenixVisibilityLevel.SNAPSHOT, state.getVisibilityLevel());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
index 17e6fb6..9f3dd59 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
@@ -23,12 +23,11 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
+import org.apache.phoenix.util.ScanUtil;
 
 public interface IndexMetaDataCache extends Closeable {
-    public static int UNKNOWN_CLIENT_VERSION = VersionUtil.encodeVersion(4, 4, 0);
     public static final IndexMetaDataCache EMPTY_INDEX_META_DATA_CACHE = new IndexMetaDataCache() {
 
         @Override
@@ -47,7 +46,7 @@ public interface IndexMetaDataCache extends Closeable {
         
         @Override
         public int getClientVersion() {
-            return UNKNOWN_CLIENT_VERSION;
+            return ScanUtil.UNKNOWN_CLIENT_VERSION;
         }
         
     };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 40917e8..5ed4130 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.cache.ServerCacheClient;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
@@ -784,8 +783,8 @@ public class DeleteCompiler {
                     byte[] uuidValue = ServerCacheClient.generateId();
                     context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                     context.getScan().setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
-                    context.getScan().setAttribute(PhoenixIndexCodec.CLIENT_VERSION, Bytes.toBytes(MetaDataProtocol.PHOENIX_VERSION));
                     context.getScan().setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
+                    ScanUtil.setClientVersion(context.getScan(), MetaDataProtocol.PHOENIX_VERSION);
                 }
                 ResultIterator iterator = aggPlan.iterator();
                 try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index c84e1d7..1341ecc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -821,7 +821,7 @@ public class FromCompiler {
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
                     MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
                     Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
-                    false, null, null, null, false, false, 0, 0L, SchemaUtil
+                    false, null, null, null, false, null, 0, 0L, SchemaUtil
                             .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
 
             String alias = subselectNode.getAlias();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 88e8f50..824d933 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -1276,7 +1276,7 @@ public class JoinCompiler {
                 left.getBucketNum(), merged, left.getParentSchemaName(), left.getParentTableName(), left.getIndexes(),
                 left.isImmutableRows(), Collections.<PName> emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL,
                 left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexId(),
-                left.getIndexType(), left.rowKeyOrderOptimizable(), left.isTransactional(),
+                left.getIndexType(), left.rowKeyOrderOptimizable(), left.getTransactionProvider(),
                 left.getUpdateCacheFrequency(), left.getIndexDisableTimestamp(), left.isNamespaceMapped(), 
                 left.getAutoPartitionSeqName(), left.isAppendOnlySchema(), ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, left.useStatsForParallelization());
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 6e52cd5..91be356 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -167,7 +167,7 @@ public class TupleProjectionCompiler {
                 table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName> emptyList(),
                 table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
                 table.getViewIndexId(),
-                table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), 
+                table.getIndexType(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(), 
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
     
@@ -198,7 +198,7 @@ public class TupleProjectionCompiler {
                 table.getBucketNum(), projectedColumns, null, null,
                 Collections.<PTable> emptyList(), table.isImmutableRows(), Collections.<PName> emptyList(), null, null,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
-                table.getViewIndexId(), null, table.rowKeyOrderOptimizable(), table.isTransactional(),
+                table.getViewIndexId(), null, table.rowKeyOrderOptimizable(), table.getTransactionProvider(),
                 table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), cqCounter, table.useStatsForParallelization());
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index d5bfef8..9ca92f9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -35,12 +35,12 @@ import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
-import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
-import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -100,7 +100,7 @@ public class UnionCompiler {
             UNION_SCHEMA_NAME, UNION_TABLE_NAME, PTableType.SUBQUERY, null,
             HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn,
             null, null, projectedColumns, null, null, null, true, null, null, null, true,
-            true, true, null, null, null, false, false, 0, 0L,
+            true, true, null, null, null, false, null, 0, 0L,
             SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
                 statement.getConnection().getQueryServices().getProps()), null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index b89e573..ab742b3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -1020,8 +1020,8 @@ public class UpsertCompiler {
                 byte[] uuidValue = ServerCacheClient.generateId();
                 scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                 scan.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
-                scan.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, Bytes.toBytes(MetaDataProtocol.PHOENIX_VERSION));
                 scan.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
+                ScanUtil.setClientVersion(scan, MetaDataProtocol.PHOENIX_VERSION);
             }
             ResultIterator iterator = aggPlan.iterator();
             try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index 3f98032..725e792 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -41,10 +41,8 @@ import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.iterate.NonAggregateRegionScannerFactory;
 import org.apache.phoenix.iterate.RegionScannerFactory;
-import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.types.PUnsignedTinyint;
-import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
 
@@ -101,6 +99,7 @@ abstract public class BaseScannerRegionObserver implements RegionObserver {
     public final static String QUALIFIER_ENCODING_SCHEME = "_QualifierEncodingScheme";
     public final static String IMMUTABLE_STORAGE_ENCODING_SCHEME = "_ImmutableStorageEncodingScheme";
     public final static String USE_ENCODED_COLUMN_QUALIFIER_LIST = "_UseEncodedColumnQualifierList";
+    public static final String CLIENT_VERSION = "_ClientVersion";
     
     public final static byte[] REPLAY_TABLE_AND_INDEX_WRITES = PUnsignedTinyint.INSTANCE.toBytes(1);
     public final static byte[] REPLAY_ONLY_INDEX_WRITES = PUnsignedTinyint.INSTANCE.toBytes(2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index f09f6ee..36e1ca1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -69,6 +69,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION_BYTES;
@@ -229,6 +230,7 @@ import org.apache.phoenix.schema.types.PTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.trace.util.Tracing;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
@@ -274,29 +276,30 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
     public static final String ROW_KEY_ORDER_OPTIMIZABLE = "ROW_KEY_ORDER_OPTIMIZABLE";
     public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
     // KeyValues for Table
-    private static final Cell TABLE_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
-    private static final Cell TABLE_SEQ_NUM_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
-    private static final Cell COLUMN_COUNT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_COUNT_BYTES);
-    private static final Cell SALT_BUCKETS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, SALT_BUCKETS_BYTES);
-    private static final Cell PK_NAME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, PK_NAME_BYTES);
-    private static final Cell DATA_TABLE_NAME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DATA_TABLE_NAME_BYTES);
-    private static final Cell INDEX_STATE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_STATE_BYTES);
-    private static final Cell IMMUTABLE_ROWS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IMMUTABLE_ROWS_BYTES);
-    private static final Cell VIEW_EXPRESSION_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_STATEMENT_BYTES);
-    private static final Cell DEFAULT_COLUMN_FAMILY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DEFAULT_COLUMN_FAMILY_NAME_BYTES);
-    private static final Cell DISABLE_WAL_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DISABLE_WAL_BYTES);
-    private static final Cell MULTI_TENANT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, MULTI_TENANT_BYTES);
-    private static final Cell VIEW_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_TYPE_BYTES);
-    private static final Cell VIEW_INDEX_ID_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_INDEX_ID_BYTES);
-    private static final Cell INDEX_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_TYPE_BYTES);
-    private static final Cell INDEX_DISABLE_TIMESTAMP_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES);
-    private static final Cell STORE_NULLS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, STORE_NULLS_BYTES);
-    private static final Cell EMPTY_KEYVALUE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES);
-    private static final Cell BASE_COLUMN_COUNT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES);
-    private static final Cell ROW_KEY_ORDER_OPTIMIZABLE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ROW_KEY_ORDER_OPTIMIZABLE_BYTES);
-    private static final Cell TRANSACTIONAL_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TRANSACTIONAL_BYTES);
-    private static final Cell UPDATE_CACHE_FREQUENCY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, UPDATE_CACHE_FREQUENCY_BYTES);
-    private static final Cell IS_NAMESPACE_MAPPED_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY,
+    private static final KeyValue TABLE_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
+    private static final KeyValue TABLE_SEQ_NUM_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
+    private static final KeyValue COLUMN_COUNT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_COUNT_BYTES);
+    private static final KeyValue SALT_BUCKETS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, SALT_BUCKETS_BYTES);
+    private static final KeyValue PK_NAME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, PK_NAME_BYTES);
+    private static final KeyValue DATA_TABLE_NAME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DATA_TABLE_NAME_BYTES);
+    private static final KeyValue INDEX_STATE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_STATE_BYTES);
+    private static final KeyValue IMMUTABLE_ROWS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IMMUTABLE_ROWS_BYTES);
+    private static final KeyValue VIEW_EXPRESSION_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_STATEMENT_BYTES);
+    private static final KeyValue DEFAULT_COLUMN_FAMILY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DEFAULT_COLUMN_FAMILY_NAME_BYTES);
+    private static final KeyValue DISABLE_WAL_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DISABLE_WAL_BYTES);
+    private static final KeyValue MULTI_TENANT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, MULTI_TENANT_BYTES);
+    private static final KeyValue VIEW_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_TYPE_BYTES);
+    private static final KeyValue VIEW_INDEX_ID_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_INDEX_ID_BYTES);
+    private static final KeyValue INDEX_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_TYPE_BYTES);
+    private static final KeyValue INDEX_DISABLE_TIMESTAMP_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES);
+    private static final KeyValue STORE_NULLS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, STORE_NULLS_BYTES);
+    private static final KeyValue EMPTY_KEYVALUE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES);
+    private static final KeyValue BASE_COLUMN_COUNT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES);
+    private static final KeyValue ROW_KEY_ORDER_OPTIMIZABLE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ROW_KEY_ORDER_OPTIMIZABLE_BYTES);
+    private static final KeyValue TRANSACTIONAL_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TRANSACTIONAL_BYTES);
+    private static final KeyValue TRANSACTION_PROVIDER_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TRANSACTION_PROVIDER_BYTES);
+    private static final KeyValue UPDATE_CACHE_FREQUENCY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, UPDATE_CACHE_FREQUENCY_BYTES);
+    private static final KeyValue IS_NAMESPACE_MAPPED_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY,
             TABLE_FAMILY_BYTES, IS_NAMESPACE_MAPPED_BYTES);
     private static final Cell AUTO_PARTITION_SEQ_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, AUTO_PARTITION_SEQ_BYTES);
     private static final Cell APPEND_ONLY_SCHEMA_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, APPEND_ONLY_SCHEMA_BYTES);
@@ -326,6 +329,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
             BASE_COLUMN_COUNT_KV,
             ROW_KEY_ORDER_OPTIMIZABLE_KV,
             TRANSACTIONAL_KV,
+            TRANSACTION_PROVIDER_KV,
             UPDATE_CACHE_FREQUENCY_KV,
             IS_NAMESPACE_MAPPED_KV,
             AUTO_PARTITION_SEQ_KV,
@@ -357,6 +361,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
     private static final int BASE_COLUMN_COUNT_INDEX = TABLE_KV_COLUMNS.indexOf(BASE_COLUMN_COUNT_KV);
     private static final int ROW_KEY_ORDER_OPTIMIZABLE_INDEX = TABLE_KV_COLUMNS.indexOf(ROW_KEY_ORDER_OPTIMIZABLE_KV);
     private static final int TRANSACTIONAL_INDEX = TABLE_KV_COLUMNS.indexOf(TRANSACTIONAL_KV);
+    private static final int TRANSACTION_PROVIDER_INDEX = TABLE_KV_COLUMNS.indexOf(TRANSACTION_PROVIDER_KV);
     private static final int UPDATE_CACHE_FREQUENCY_INDEX = TABLE_KV_COLUMNS.indexOf(UPDATE_CACHE_FREQUENCY_KV);
     private static final int INDEX_DISABLE_TIMESTAMP = TABLE_KV_COLUMNS.indexOf(INDEX_DISABLE_TIMESTAMP_KV);
     private static final int IS_NAMESPACE_MAPPED_INDEX = TABLE_KV_COLUMNS.indexOf(IS_NAMESPACE_MAPPED_KV);
@@ -950,7 +955,24 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
         Cell storeNullsKv = tableKeyValues[STORE_NULLS_INDEX];
         boolean storeNulls = storeNullsKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(storeNullsKv.getValueArray(), storeNullsKv.getValueOffset(), storeNullsKv.getValueLength()));
         Cell transactionalKv = tableKeyValues[TRANSACTIONAL_INDEX];
-        boolean transactional = transactionalKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(transactionalKv.getValueArray(), transactionalKv.getValueOffset(), transactionalKv.getValueLength()));
+        Cell transactionProviderKv = tableKeyValues[TRANSACTION_PROVIDER_INDEX];
+        TransactionFactory.Provider transactionProvider = null;
+        if (transactionProviderKv == null) {
+            if (transactionalKv != null && Boolean.TRUE.equals(
+                    PBoolean.INSTANCE.toObject(
+                            transactionalKv.getValueArray(), 
+                            transactionalKv.getValueOffset(), 
+                            transactionalKv.getValueLength()))) {
+                // For backward compat, prior to client setting TRANSACTION_PROVIDER
+                transactionProvider = TransactionFactory.Provider.TEPHRA;
+            }
+        } else {
+            transactionProvider = TransactionFactory.Provider.fromCode(
+                    PTinyint.INSTANCE.getCodec().decodeByte(
+                            transactionProviderKv.getValueArray(),
+                            transactionProviderKv.getValueOffset(), 
+                            SortOrder.getDefault()));
+        }
         Cell viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
         ViewType viewType = viewTypeKv == null ? null : ViewType.fromSerializedValue(viewTypeKv.getValueArray()[viewTypeKv.getValueOffset()]);
         Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
@@ -1033,7 +1055,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements RegionCopr
         return PTableImpl.makePTable(tenantId, schemaName, tableName, tableType, indexState, timeStamp, tableSeqNum,
                 pkName, saltBucketNum, columns, parentSchemaName, parentTableName, indexes, isImmutableRows, physicalTables, defaultFamilyName,
                 viewStatement, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType,
-                rowKeyOrderOptimizable, transactional, updateCacheFrequency, baseColumnCount,
+                rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, baseColumnCount,
                 indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, encodingScheme, cqCounter, useStatsForParallelization);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 9de9fba..906f58f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -95,9 +95,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
     // Since there's no upgrade code, keep the version the same as the previous version
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0 = MIN_TABLE_TIMESTAMP + 28;
     // TODO Was there a system table upgrade?
     // TODO Need to account for the inevitable 4.14 release too
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_5_0_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_5_0_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_5_0_0;
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index 116b160..83da368 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -85,6 +84,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.UpgradeUtil;
@@ -486,7 +486,7 @@ public class MetaDataRegionObserver implements RegionObserver,RegionCoprocessor
 									conn);
 							byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(indexMetaDataPtr);
 							dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
-							dataTableScan.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, Bytes.toBytes(MetaDataProtocol.PHOENIX_VERSION));
+							ScanUtil.setClientVersion(dataTableScan, MetaDataProtocol.PHOENIX_VERSION);
                             LOG.info("Starting to partially build indexes:" + indexesToPartiallyRebuild
                                     + " on data table:" + dataPTable.getName() + " with the earliest disable timestamp:"
                                     + earliestDisableTimestamp + " till "

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixTransactionalProcessor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixTransactionalProcessor.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixTransactionalProcessor.java
deleted file mode 100644
index 8e7f0f9..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixTransactionalProcessor.java
+++ /dev/null
@@ -1,52 +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.phoenix.coprocessor;
-
-import java.io.IOException;
-import java.util.Optional;
-
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.phoenix.transaction.TransactionFactory;
-
-public class PhoenixTransactionalProcessor extends DelegateRegionObserver implements RegionCoprocessor {
-
-    @Override
-    public Optional<RegionObserver> getRegionObserver() {
-        return Optional.of(this);
-    }
-
-    public PhoenixTransactionalProcessor() {
-        super(TransactionFactory.getTransactionProvider().getTransactionContext().getCoprocessor());
-    }
-
-    @Override
-    public void start(CoprocessorEnvironment env) throws IOException {
-        if (delegate instanceof RegionCoprocessor) {
-            ((RegionCoprocessor)delegate).start(env);
-        }
-    }
-
-    @Override
-    public void stop(CoprocessorEnvironment env) throws IOException {
-        if (delegate instanceof RegionCoprocessor) {
-            ((RegionCoprocessor)delegate).stop(env);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
index b586d23..46b847b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.cache.GlobalCache;
-import org.apache.phoenix.cache.IndexMetaDataCache;
 import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.AddServerCacheRequest;
@@ -37,6 +36,7 @@ import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ServerCachin
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
 
 import com.google.protobuf.RpcCallback;
@@ -80,7 +80,7 @@ public class ServerCachingEndpointImpl extends ServerCachingService implements R
           ServerCacheFactory cacheFactory = serverCacheFactoryClass.newInstance();
           tenantCache.addServerCache(new ImmutableBytesPtr(request.getCacheId().toByteArray()),
               cachePtr, txState, cacheFactory, request.hasHasProtoBufIndexMaintainer() && request.getHasProtoBufIndexMaintainer(),
-              request.hasClientVersion() ? request.getClientVersion() : IndexMetaDataCache.UNKNOWN_CLIENT_VERSION);
+              request.hasClientVersion() ? request.getClientVersion() : ScanUtil.UNKNOWN_CLIENT_VERSION);
         } catch (Throwable e) {
             ProtobufUtil.setControllerException(controller,
                 ServerUtil.createIOException("Error when adding cache: ", e));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TephraTransactionalProcessor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TephraTransactionalProcessor.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TephraTransactionalProcessor.java
new file mode 100644
index 0000000..712904a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TephraTransactionalProcessor.java
@@ -0,0 +1,52 @@
+/*
+ * 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.phoenix.coprocessor;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+
+public class TephraTransactionalProcessor extends DelegateRegionObserver implements RegionCoprocessor {
+
+    public TephraTransactionalProcessor() {
+        super(new TransactionProcessor());
+    }
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+        return Optional.of(this);
+    }
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+        if (delegate instanceof RegionCoprocessor) {
+            ((RegionCoprocessor)delegate).start(env);
+        }
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+        if (delegate instanceof RegionCoprocessor) {
+            ((RegionCoprocessor)delegate).stop(env);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 37d2b4d..3fe1def 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -279,7 +279,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                m.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
            }
            if (clientVersionBytes != null) {
-               m.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, clientVersionBytes);
+               m.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersionBytes);
            }
         }
     }
@@ -526,7 +526,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             useIndexProto = false;
         }
 
-        byte[] clientVersionBytes = scan.getAttribute(PhoenixIndexCodec.CLIENT_VERSION);
+        byte[] clientVersionBytes = scan.getAttribute(BaseScannerRegionObserver.CLIENT_VERSION);
         boolean acquiredLock = false;
         boolean incrScanRefCount = false;
         final TenantCache tenantCache = GlobalCache.getTenantCache(env, ScanUtil.getTenantId(scan));
@@ -985,7 +985,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             useProto = false;
             indexMetaData = scan.getAttribute(PhoenixIndexCodec.INDEX_MD);
         }
-        byte[] clientVersionBytes = scan.getAttribute(PhoenixIndexCodec.CLIENT_VERSION);
+        byte[] clientVersionBytes = scan.getAttribute(BaseScannerRegionObserver.CLIENT_VERSION);
         boolean hasMore;
         int rowCount = 0;
         try {
@@ -1010,7 +1010,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     put.setAttribute(useProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMetaData);
                                     put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                                     put.setAttribute(REPLAY_WRITES, REPLAY_ONLY_INDEX_WRITES);
-                                    put.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, clientVersionBytes);
+                                    put.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersionBytes);
                                     mutations.add(put);
                                     // Since we're replaying existing mutations, it makes no sense to write them to the wal
                                     put.setDurability(Durability.SKIP_WAL);
@@ -1022,7 +1022,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     del.setAttribute(useProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMetaData);
                                     del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                                     del.setAttribute(REPLAY_WRITES, REPLAY_ONLY_INDEX_WRITES);
-                                    del.setAttribute(PhoenixIndexCodec.CLIENT_VERSION, clientVersionBytes);
+                                    del.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersionBytes);
                                     mutations.add(del);
                                     // Since we're replaying existing mutations, it makes no sense to write them to the wal
                                     del.setDurability(Durability.SKIP_WAL);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 278f301..8d500e8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -3520,6 +3520,16 @@ public final class PTableProtos {
      * <code>optional bool useStatsForParallelization = 37;</code>
      */
     boolean getUseStatsForParallelization();
+
+    // optional int32 transactionProvider = 38;
+    /**
+     * <code>optional int32 transactionProvider = 38;</code>
+     */
+    boolean hasTransactionProvider();
+    /**
+     * <code>optional int32 transactionProvider = 38;</code>
+     */
+    int getTransactionProvider();
   }
   /**
    * Protobuf type {@code PTable}
@@ -3771,6 +3781,11 @@ public final class PTableProtos {
               useStatsForParallelization_ = input.readBool();
               break;
             }
+            case 304: {
+              bitField1_ |= 0x00000001;
+              transactionProvider_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3823,6 +3838,7 @@ public final class PTableProtos {
     }
 
     private int bitField0_;
+    private int bitField1_;
     // required bytes schemaNameBytes = 1;
     public static final int SCHEMANAMEBYTES_FIELD_NUMBER = 1;
     private com.google.protobuf.ByteString schemaNameBytes_;
@@ -4534,6 +4550,22 @@ public final class PTableProtos {
       return useStatsForParallelization_;
     }
 
+    // optional int32 transactionProvider = 38;
+    public static final int TRANSACTIONPROVIDER_FIELD_NUMBER = 38;
+    private int transactionProvider_;
+    /**
+     * <code>optional int32 transactionProvider = 38;</code>
+     */
+    public boolean hasTransactionProvider() {
+      return ((bitField1_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 transactionProvider = 38;</code>
+     */
+    public int getTransactionProvider() {
+      return transactionProvider_;
+    }
+
     private void initFields() {
       schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       tableNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -4571,6 +4603,7 @@ public final class PTableProtos {
       encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
       encodedCQCounters_ = java.util.Collections.emptyList();
       useStatsForParallelization_ = false;
+      transactionProvider_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4746,6 +4779,9 @@ public final class PTableProtos {
       if (((bitField0_ & 0x80000000) == 0x80000000)) {
         output.writeBool(37, useStatsForParallelization_);
       }
+      if (((bitField1_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(38, transactionProvider_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4904,6 +4940,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(37, useStatsForParallelization_);
       }
+      if (((bitField1_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(38, transactionProvider_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -5095,6 +5135,11 @@ public final class PTableProtos {
         result = result && (getUseStatsForParallelization()
             == other.getUseStatsForParallelization());
       }
+      result = result && (hasTransactionProvider() == other.hasTransactionProvider());
+      if (hasTransactionProvider()) {
+        result = result && (getTransactionProvider()
+            == other.getTransactionProvider());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -5252,6 +5297,10 @@ public final class PTableProtos {
         hash = (37 * hash) + USESTATSFORPARALLELIZATION_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getUseStatsForParallelization());
       }
+      if (hasTransactionProvider()) {
+        hash = (37 * hash) + TRANSACTIONPROVIDER_FIELD_NUMBER;
+        hash = (53 * hash) + getTransactionProvider();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5448,6 +5497,8 @@ public final class PTableProtos {
         }
         useStatsForParallelization_ = false;
         bitField1_ = (bitField1_ & ~0x00000008);
+        transactionProvider_ = 0;
+        bitField1_ = (bitField1_ & ~0x00000010);
         return this;
       }
 
@@ -5477,6 +5528,7 @@ public final class PTableProtos {
         int from_bitField0_ = bitField0_;
         int from_bitField1_ = bitField1_;
         int to_bitField0_ = 0;
+        int to_bitField1_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
@@ -5637,7 +5689,12 @@ public final class PTableProtos {
           to_bitField0_ |= 0x80000000;
         }
         result.useStatsForParallelization_ = useStatsForParallelization_;
+        if (((from_bitField1_ & 0x00000010) == 0x00000010)) {
+          to_bitField1_ |= 0x00000001;
+        }
+        result.transactionProvider_ = transactionProvider_;
         result.bitField0_ = to_bitField0_;
+        result.bitField1_ = to_bitField1_;
         onBuilt();
         return result;
       }
@@ -5841,6 +5898,9 @@ public final class PTableProtos {
         if (other.hasUseStatsForParallelization()) {
           setUseStatsForParallelization(other.getUseStatsForParallelization());
         }
+        if (other.hasTransactionProvider()) {
+          setTransactionProvider(other.getTransactionProvider());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7920,6 +7980,39 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int32 transactionProvider = 38;
+      private int transactionProvider_ ;
+      /**
+       * <code>optional int32 transactionProvider = 38;</code>
+       */
+      public boolean hasTransactionProvider() {
+        return ((bitField1_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional int32 transactionProvider = 38;</code>
+       */
+      public int getTransactionProvider() {
+        return transactionProvider_;
+      }
+      /**
+       * <code>optional int32 transactionProvider = 38;</code>
+       */
+      public Builder setTransactionProvider(int value) {
+        bitField1_ |= 0x00000010;
+        transactionProvider_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 transactionProvider = 38;</code>
+       */
+      public Builder clearTransactionProvider() {
+        bitField1_ = (bitField1_ & ~0x00000010);
+        transactionProvider_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PTable)
     }
 
@@ -8587,7 +8680,7 @@ public final class PTableProtos {
       "\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006values\030",
       "\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n\rke" +
       "yBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001" +
-      "(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"\220\007" +
+      "(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"\255\007" +
       "\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tab" +
       "leNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.P" +
       "TableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenc" +
@@ -8610,12 +8703,13 @@ public final class PTableProtos {
       "\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageScheme" +
       "\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021encode" +
       "dCQCounters\030$ \003(\0132\021.EncodedCQCounter\022\"\n\032" +
-      "useStatsForParallelization\030% \001(\010\"6\n\020Enco" +
-      "dedCQCounter\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007count" +
-      "er\030\002 \002(\005*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004US" +
-      "ER\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(o" +
-      "rg.apache.phoenix.coprocessor.generatedB",
-      "\014PTableProtosH\001\210\001\001\240\001\001"
+      "useStatsForParallelization\030% \001(\010\022\033\n\023tran" +
+      "sactionProvider\030& \001(\005\"6\n\020EncodedCQCounte" +
+      "r\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A\n" +
+      "\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIE" +
+      "W\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.p",
+      "hoenix.coprocessor.generatedB\014PTableProt" +
+      "osH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8639,7 +8733,7 @@ public final class PTableProtos {
           internal_static_PTable_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PTable_descriptor,
-              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisableTimestamp", "IsNamespaceMapped", "AutoParititonSeqName", "IsAppendOnlySchema", "ParentNameBytes", "StorageScheme", "EncodingScheme", "EncodedCQCounters", "UseStatsForParallelization", });
+              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisableTimestamp", "IsNamespaceMapped", "AutoParititonSeqName", "IsAppendOnlySchema", "ParentNameBytes", "StorageScheme", "EncodingScheme", "EncodedCQCounters", "UseStatsForParallelization", "TransactionProvider", });
           internal_static_EncodedCQCounter_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_EncodedCQCounter_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 61e1307..bde5083 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -294,6 +294,10 @@ public enum SQLExceptionCode {
     SEQUENCE_NOT_CASTABLE_TO_AUTO_PARTITION_ID_COLUMN(1086, "44A17", "Sequence Value not castable to auto-partition id column"),
     CANNOT_COERCE_AUTO_PARTITION_ID(1087, "44A18", "Auto-partition id cannot be coerced"),
     CANNOT_CREATE_INDEX_ON_MUTABLE_TABLE_WITH_ROWTIMESTAMP(1088, "44A19", "Cannot create an index on a mutable table that has a ROW_TIMESTAMP column."),
+    UNKNOWN_TRANSACTION_PROVIDER(1089,"44A20", "Unknown TRANSACTION_PROVIDER: "),
+    CANNOT_START_TXN_IF_TXN_DISABLED(1091, "44A22", "Cannot start transaction if transactions are disabled."),
+    CANNOT_MIX_TXN_PROVIDERS(1092, "44A23", "Cannot mix transaction providers: "),
+    CANNOT_ALTER_TABLE_FROM_NON_TXN_TO_TXNL(1093, "44A24", "Cannot alter table from non transactional to transactional for "),
 
     /** Sequence related */
     SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new Factory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index b152030..8a8c822 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -47,6 +47,7 @@ import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.WhereCompiler;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -252,6 +253,8 @@ public abstract class BaseQueryPlan implements QueryPlan {
             return newIterator(scanGrouper, scan, caches);
         }
         
+        ScanUtil.setClientVersion(scan, MetaDataProtocol.PHOENIX_VERSION);
+        
         // Set miscellaneous scan attributes. This is the last chance to set them before we
         // clone the scan for each parallelized chunk.
         TableRef tableRef = context.getCurrentTable();