You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2018/04/14 07:37:17 UTC

[01/21] phoenix git commit: PHOENIX-4683 Cap timeouts for stats precompact hook logic [Forced Update!]

Repository: phoenix
Updated Branches:
  refs/heads/4.x-cdh5.12 031ca5a1f -> 157139688 (forced update)


PHOENIX-4683 Cap timeouts for stats precompact hook logic


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

Branch: refs/heads/4.x-cdh5.12
Commit: 7871e72c95c4966e6dcc81b02cab65ad29a69bff
Parents: f3defc4
Author: Vincent Poon <vi...@apache.org>
Authored: Mon Apr 9 22:04:28 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:24:49 2018 +0100

----------------------------------------------------------------------
 .../DelegateRegionCoprocessorEnvironment.java   |  9 ++-
 .../UngroupedAggregateRegionObserver.java       |  8 ++-
 .../hbase/index/write/IndexWriterUtils.java     | 71 ++-----------------
 .../org/apache/phoenix/util/ServerUtil.java     | 72 ++++++++++++++++++++
 4 files changed, 89 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7871e72c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
index 380212e..284d53c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.util.ServerUtil;
 
 /**
  * Class to encapsulate {@link RegionCoprocessorEnvironment} for phoenix coprocessors. Often we
@@ -39,10 +42,12 @@ public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEn
 
     private final Configuration config;
     private RegionCoprocessorEnvironment delegate;
+    private HTableFactory tableFactory;
 
     public DelegateRegionCoprocessorEnvironment(Configuration config, RegionCoprocessorEnvironment delegate) {
         this.config = config;
         this.delegate = delegate;
+        this.tableFactory = ServerUtil.getDelegateHTableFactory(this, config);
     }
 
     @Override
@@ -77,13 +82,13 @@ public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEn
 
     @Override
     public HTableInterface getTable(TableName tableName) throws IOException {
-        return delegate.getTable(tableName);
+        return tableFactory.getTable(new ImmutableBytesPtr(tableName.getName()));
     }
 
     @Override
     public HTableInterface getTable(TableName tableName, ExecutorService service)
             throws IOException {
-        return delegate.getTable(tableName, service);
+        return tableFactory.getTable(new ImmutableBytesPtr(tableName.getName()));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7871e72c/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 965ba1b..27d3880 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
@@ -94,6 +94,7 @@ import org.apache.phoenix.hbase.index.exception.IndexWriteException;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
+import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.index.PhoenixIndexFailurePolicy;
@@ -978,10 +979,13 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                     InternalScanner internalScanner = scanner;
                     try {
                         long clientTimeStamp = EnvironmentEdgeManager.currentTimeMillis();
+                        DelegateRegionCoprocessorEnvironment compactionConfEnv = new DelegateRegionCoprocessorEnvironment(compactionConfig, c.getEnvironment());
                         StatisticsCollector stats = StatisticsCollectorFactory.createStatisticsCollector(
-                            c.getEnvironment(), table.getNameAsString(), clientTimeStamp,
+                            compactionConfEnv, table.getNameAsString(), clientTimeStamp,
                             store.getFamily().getName());
-                        internalScanner = stats.createCompactionScanner(c.getEnvironment(), store, scanner);
+                        internalScanner =
+                                stats.createCompactionScanner(compactionConfEnv,
+                                    store, scanner);
                     } catch (Exception e) {
                         // If we can't reach the stats table, don't interrupt the normal
                         // compaction operation, just log a warning.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7871e72c/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
index 29b9faf..76d6800 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
@@ -17,25 +17,14 @@
  */
 package org.apache.phoenix.hbase.index.write;
 
-import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.concurrent.GuardedBy;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.client.CoprocessorHConnection;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.phoenix.hbase.index.table.CoprocessorHTableFactory;
 import org.apache.phoenix.hbase.index.table.HTableFactory;
-import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ServerUtil;
 
 public class IndexWriterUtils {
 
@@ -86,66 +75,14 @@ public class IndexWriterUtils {
 
     public static HTableFactory getDefaultDelegateHTableFactory(CoprocessorEnvironment env) {
         // create a simple delegate factory, setup the way we need
-        Configuration conf = env.getConfiguration();
+        Configuration conf = PropertiesUtil.cloneConfig(env.getConfiguration());
         // set the number of threads allowed per table.
         int htableThreads =
                 conf.getInt(IndexWriterUtils.INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY,
                     IndexWriterUtils.DEFAULT_NUM_PER_TABLE_THREADS);
         LOG.trace("Creating HTableFactory with " + htableThreads + " threads for each HTable.");
         IndexManagementUtil.setIfNotSet(conf, HTABLE_THREAD_KEY, htableThreads);
-        if (env instanceof RegionCoprocessorEnvironment) {
-            RegionCoprocessorEnvironment e = (RegionCoprocessorEnvironment) env;
-            RegionServerServices services = e.getRegionServerServices();
-            if (services instanceof HRegionServer) {
-                return new CoprocessorHConnectionTableFactory(conf, (HRegionServer) services);
-            }
-        }
-        return new CoprocessorHTableFactory(env);
+        return ServerUtil.getDelegateHTableFactory(env, conf);
     }
 
-    /**
-     * {@code HTableFactory} that creates HTables by using a {@link CoprocessorHConnection} This
-     * factory was added as a workaround to the bug reported in
-     * https://issues.apache.org/jira/browse/HBASE-18359
-     */
-    private static class CoprocessorHConnectionTableFactory implements HTableFactory {
-        @GuardedBy("CoprocessorHConnectionTableFactory.this")
-        private HConnection connection;
-        private final Configuration conf;
-        private final HRegionServer server;
-
-        CoprocessorHConnectionTableFactory(Configuration conf, HRegionServer server) {
-            this.conf = conf;
-            this.server = server;
-        }
-
-        private synchronized HConnection getConnection(Configuration conf) throws IOException {
-            if (connection == null || connection.isClosed()) {
-                connection = new CoprocessorHConnection(conf, server);
-            }
-            return connection;
-        }
-
-        @Override
-        public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
-            return getConnection(conf).getTable(tablename.copyBytesIfNecessary());
-        }
-
-        @Override
-        public synchronized void shutdown() {
-            try {
-                if (connection != null && !connection.isClosed()) {
-                    connection.close();
-                }
-            } catch (Throwable e) {
-                LOG.warn("Error while trying to close the HConnection used by CoprocessorHConnectionTableFactory", e);
-            }
-        }
-
-        @Override
-        public HTableInterface getTable(ImmutableBytesPtr tablename, ExecutorService pool)
-                throws IOException {
-            return getConnection(conf).getTable(tablename.copyBytesIfNecessary(), pool);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7871e72c/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
index bc2b625..4b3cc43 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
@@ -21,29 +21,44 @@ import java.io.IOException;
 import java.sql.SQLException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import javax.annotation.concurrent.GuardedBy;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CoprocessorHConnection;
+import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.HashJoinCacheNotFoundException;
 import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.hbase.index.table.CoprocessorHTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 
 
 @SuppressWarnings("deprecation")
 public class ServerUtil {
+    private static final Log LOG = LogFactory.getLog(ServerUtil.class);
     private static final int COPROCESSOR_SCAN_WORKS = VersionUtil.encodeVersion("0.98.6");
     
     private static final String FORMAT = "ERROR %d (%s): %s";
@@ -254,4 +269,61 @@ public class ServerUtil {
                     endKey) < 0));
     }
 
+    public static HTableFactory getDelegateHTableFactory(CoprocessorEnvironment env, Configuration conf) {
+        if (env instanceof RegionCoprocessorEnvironment) {
+            RegionCoprocessorEnvironment e = (RegionCoprocessorEnvironment) env;
+            RegionServerServices services = e.getRegionServerServices();
+            if (services instanceof HRegionServer) {
+                return new CoprocessorHConnectionTableFactory(conf, (HRegionServer) services);
+            }
+        }
+        return new CoprocessorHTableFactory(env);
+    }
+
+    /**
+     * {@code HTableFactory} that creates HTables by using a {@link CoprocessorHConnection} This
+     * factory was added as a workaround to the bug reported in
+     * https://issues.apache.org/jira/browse/HBASE-18359
+     */
+    public static class CoprocessorHConnectionTableFactory implements HTableFactory {
+        @GuardedBy("CoprocessorHConnectionTableFactory.this")
+        private HConnection connection;
+        private final Configuration conf;
+        private final HRegionServer server;
+
+        CoprocessorHConnectionTableFactory(Configuration conf, HRegionServer server) {
+            this.conf = conf;
+            this.server = server;
+        }
+
+        private synchronized HConnection getConnection(Configuration conf) throws IOException {
+            if (connection == null || connection.isClosed()) {
+                connection = new CoprocessorHConnection(conf, server);
+            }
+            return connection;
+        }
+
+        @Override
+        public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
+            return getConnection(conf).getTable(tablename.copyBytesIfNecessary());
+        }
+
+        @Override
+        public synchronized void shutdown() {
+            try {
+                if (connection != null && !connection.isClosed()) {
+                    connection.close();
+                }
+            } catch (Throwable e) {
+                LOG.warn("Error while trying to close the HConnection used by CoprocessorHConnectionTableFactory", e);
+            }
+        }
+
+        @Override
+        public HTableInterface getTable(ImmutableBytesPtr tablename, ExecutorService pool)
+                throws IOException {
+            return getConnection(conf).getTable(tablename.copyBytesIfNecessary(), pool);
+        }
+    }
+
 }


[18/21] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

Posted by pb...@apache.org.
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 8cda81411294ead4b5642fca842c9c56a15e8652
Parents: f71654a
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Apr 13 18:30:30 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:31:28 2018 +0100

----------------------------------------------------------------------
 .../phoenix/end2end/AppendOnlySchemaIT.java     |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java    | 626 ++++++++++++++++
 .../coprocessor/MetaDataEndpointImpl.java       |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java     |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java      | 750 +++++++++++--------
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1236 insertions(+), 391 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT {
             // verify no create table rpcs
             verify(connectionQueryServices, never()).createTable(anyListOf(Mutation.class),
                 any(byte[].class), any(PTableType.class), anyMap(), anyList(), any(byte[][].class),
-                eq(false), eq(false));
+                eq(false), eq(false), eq(false));
             reset(connectionQueryServices);
             
             // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index d20ffa9..b0f1d5f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -377,8 +377,10 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
         while(rs.next()) {
 
             if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+                // Check that entry for SYSTEM namespace exists in SYSCAT
                 systemSchemaExists = rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) ? true : systemSchemaExists;
             } else if (rs.getString("COLUMN_NAME") == null) {
+                // Found the intial entry for a table in SYSCAT
                 String schemaName = rs.getString("TABLE_SCHEM");
                 String tableName = rs.getString("TABLE_NAME");
 
@@ -396,12 +398,11 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
             }
         }
 
-        if(!systemSchemaExists) {
-            fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't exist in SYSTEM.CATALOG table.");
-        }
-
         // The set will contain SYSMUTEX table since that table is not exposed in SYSCAT
         if (systemTablesMapped) {
+            if (!systemSchemaExists) {
+                fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't exist in SYSTEM.CATALOG table.");
+            }
             assertTrue(namespaceMappedSystemTablesSet.size() == 1);
         } else {
             assertTrue(systemTablesSet.size() == 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
new file mode 100644
index 0000000..689eb20
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -0,0 +1,626 @@
+/*
+ * 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.end2end;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.UpgradeRequiredException;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.*;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.*;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.*;
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class SystemCatalogCreationOnConnectionIT {
+    private HBaseTestingUtility testUtil = null;
+    private Set<String> hbaseTables;
+    private static boolean setOldTimestampToInduceUpgrade = false;
+    private static int countUpgradeAttempts;
+    // This flag is used to figure out if the SYSCAT schema was actually upgraded or not, based on the timestamp of SYSCAT
+    // (different from an upgrade attempt)
+    private static int actualSysCatUpgrades;
+    private static final String PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG = "SYSTEM:CATALOG";
+    private static final String PHOENIX_SYSTEM_CATALOG = "SYSTEM.CATALOG";
+    private static final String EXECUTE_UPGRADE_COMMAND = "EXECUTE UPGRADE";
+    private static final String MODIFIED_MAX_VERSIONS ="5";
+
+    private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
+      "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
+      "SYSTEM.MUTEX", "SYSTEM.LOG"));
+
+    private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
+      Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
+        "SYSTEM:MUTEX", "SYSTEM:LOG"));
+
+    private static class PhoenixSysCatCreationServices extends ConnectionQueryServicesImpl {
+
+        public PhoenixSysCatCreationServices(QueryServices services, PhoenixEmbeddedDriver.ConnectionInfo connectionInfo, Properties info) {
+            super(services, connectionInfo, info);
+        }
+
+        @Override
+        protected void setUpgradeRequired() {
+            super.setUpgradeRequired();
+            countUpgradeAttempts++;
+        }
+
+        @Override
+        protected long getSystemTableVersion() {
+            if (setOldTimestampToInduceUpgrade) {
+                // Return the next lower version where an upgrade was performed to induce setting the upgradeRequired flag
+                return MetaDataProtocol.getPriorUpgradeVersion();
+            }
+            return MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP;
+        }
+
+        @Override
+        protected PhoenixConnection upgradeSystemCatalogIfRequired(PhoenixConnection metaConnection,
+          long currentServerSideTableTimeStamp) throws InterruptedException, SQLException, TimeoutException, IOException {
+            PhoenixConnection newMetaConnection = super.upgradeSystemCatalogIfRequired(metaConnection, currentServerSideTableTimeStamp);
+            if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP) {
+                actualSysCatUpgrades++;
+            }
+            return newMetaConnection;
+        }
+    }
+
+    public static class PhoenixSysCatCreationTestingDriver extends PhoenixTestDriver {
+        private ConnectionQueryServices cqs;
+        private final ReadOnlyProps overrideProps;
+
+        public PhoenixSysCatCreationTestingDriver(ReadOnlyProps props) {
+            overrideProps = props;
+        }
+
+        @Override // public for testing
+        public synchronized ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
+            if (cqs == null) {
+                cqs = new PhoenixSysCatCreationServices(new QueryServicesTestImpl(getDefaultProps(), overrideProps), ConnectionInfo.create(url), info);
+                cqs.init(url, info);
+            }
+            return cqs;
+        }
+
+        // NOTE: Do not use this if you want to try re-establishing a connection from the client using a previously
+        // used ConnectionQueryServices instance. This is used only in cases where we need to test server-side
+        // changes and don't care about client-side properties set from the init method.
+        // Reset the Connection Query Services instance so we can create a new connection to the cluster
+        public void resetCQS() {
+            cqs = null;
+        }
+    }
+
+
+    @Before
+    public void resetVariables() {
+        setOldTimestampToInduceUpgrade = false;
+        countUpgradeAttempts = 0;
+        actualSysCatUpgrades = 0;
+    }
+
+    @After
+    public void tearDownMiniCluster() {
+        try {
+            if (testUtil != null) {
+                testUtil.shutdownMiniCluster();
+                testUtil = null;
+            }
+        } catch (Exception e) {
+            // ignore
+        }
+    }
+
+
+     // Conditions: isDoNotUpgradePropSet is true
+     // Expected: We do not create SYSTEM.CATALOG even if this is the first connection to the server
+    @Test
+    public void testFirstConnectionDoNotUpgradePropSet() throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.FALSE.toString());
+        Properties propsDoNotUpgradePropSet = new Properties();
+        // Set doNotUpgradeProperty to true
+        UpgradeUtil.doNotUpgradeOnFirstConnection(propsDoNotUpgradePropSet);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), propsDoNotUpgradePropSet);
+            fail("Client should not be able to create SYSTEM.CATALOG since we set the doNotUpgrade property");
+        } catch (Exception e) {
+            assertTrue(e instanceof UpgradeRequiredException);
+        }
+        hbaseTables = getHBaseTables();
+        assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG) || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 0);
+        assertEquals(1, countUpgradeAttempts);
+    }
+
+
+    /********************* Testing SYSTEM.CATALOG/SYSTEM:CATALOG creation/upgrade behavior for subsequent connections *********************/
+
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create unmapped
+    // SYSTEM tables i.e. SYSTEM\..*, the second connection has client-side namespace mapping enabled and
+    // system table to system namespace mapping enabled
+    // Expected: We will migrate all SYSTEM\..* tables to the SYSTEM namespace
+    @Test
+    public void testMigrateToSystemNamespace() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientEnabledMappingDisabled();
+        driver.resetCQS();
+        // Setting this to true to effect migration of SYSTEM tables to the SYSTEM namespace
+        Properties clientProps = getClientProperties(true, true);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
+        assertEquals(1, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create all namespace
+    // mapped SYSTEM tables i.e. SYSTEM:.*, the SYSTEM:CATALOG timestamp at creation is purposefully set to be <
+    // MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP. The subsequent connection has client-side namespace mapping enabled
+    // Expected: An upgrade is attempted when the second client connects to the server
+    @Test
+    public void testUpgradeAttempted() throws Exception {
+        setOldTimestampToInduceUpgrade = true;
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientEnabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(true, true);
+        setOldTimestampToInduceUpgrade = false;
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        // There should be no new tables
+        assertEquals(hbaseTables, getHBaseTables());
+        // Since we set an old timestamp on purpose when creating SYSTEM:CATALOG, the second connection attempts to upgrade it
+        assertEquals(1, countUpgradeAttempts);
+        assertEquals(1, actualSysCatUpgrades);
+    }
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create all namespace
+    // mapped SYSTEM tables i.e. SYSTEM:.*, the SYSTEM:CATALOG timestamp at creation is purposefully set to be <
+    // MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP. The subsequent connection has client-side namespace mapping enabled
+    // Expected: An upgrade is attempted when the second client connects to the server, but this fails since the
+    // isDoNotUpgradePropSet is set to true. We later run EXECUTE UPGRADE manually
+    @Test
+    public void testUpgradeNotAllowed() throws Exception {
+        setOldTimestampToInduceUpgrade = true;
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientEnabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(true, true);
+        UpgradeUtil.doNotUpgradeOnFirstConnection(clientProps);
+        setOldTimestampToInduceUpgrade = false;
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        } catch (Exception e) {
+            assertTrue(e instanceof UpgradeRequiredException);
+        }
+        // There should be no new tables
+        assertEquals(hbaseTables, getHBaseTables());
+        // Since we set an old timestamp on purpose when creating SYSTEM:CATALOG, the second connection attempts to upgrade it
+        assertEquals(1, countUpgradeAttempts);
+        // This connection is unable to actually upgrade SYSTEM:CATALOG due to isDoNotUpgradePropSet
+        assertEquals(0, actualSysCatUpgrades);
+        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), new Properties()).connect(getJdbcUrl(), new Properties());
+        try {
+            conn.createStatement().execute(EXECUTE_UPGRADE_COMMAND);
+            // Actually upgraded SYSTEM:CATALOG
+            assertEquals(1, actualSysCatUpgrades);
+        } finally {
+            conn.close();
+        }
+    }
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create only SYSTEM.CATALOG,
+    // the second connection has client-side namespace mapping enabled
+    // Expected: We will migrate SYSTEM.CATALOG to SYSTEM namespace and create all other SYSTEM:.* tables
+    @Test
+    public void testMigrateSysCatCreateOthers() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientDisabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(true, true);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
+        // SYSTEM.CATALOG migration to the SYSTEM namespace is counted as an upgrade
+        assertEquals(1, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create unmapped SYSTEM
+    // tables SYSTEM\..* whose timestamp at creation is purposefully set to be < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP.
+    // The second connection has client-side namespace mapping enabled and system table to system namespace mapping enabled
+    // Expected: We will migrate all SYSTEM\..* tables to the SYSTEM namespace and also upgrade SYSTEM:CATALOG
+    @Test
+    public void testMigrateToSystemNamespaceAndUpgradeSysCat() throws Exception {
+        setOldTimestampToInduceUpgrade = true;
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientEnabledMappingDisabled();
+        driver.resetCQS();
+        setOldTimestampToInduceUpgrade = false;
+        Properties clientProps = getClientProperties(true, true);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
+        assertEquals(1, countUpgradeAttempts);
+        assertEquals(1, actualSysCatUpgrades);
+    }
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create all namespace
+    // mapped SYSTEM tables i.e. SYSTEM:.*, the second connection has client-side namespace mapping disabled
+    // Expected: Throw Inconsistent namespace mapping exception from ensureTableCreated
+    @Test
+    public void testTablesExistInconsistentNSMappingFails() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientEnabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(false, false);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
+        assertEquals(0, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create only SYSTEM.CATALOG,
+    // the second connection has client-side namespace mapping disabled
+    // Expected: Throw Inconsistent namespace mapping exception when you check client-server compatibility
+    @Test
+    public void testUnmappedSysCatExistsInconsistentNSMappingFails() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerEnabledClientDisabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(false, false);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertTrue(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create all unmapped
+    // SYSTEM tables i.e. SYSTEM\..*, the second connection has client-side namespace mapping enabled
+    // Expected: Throw Inconsistent namespace mapping exception when you check client-server compatibility
+    @Test
+    public void testSysTablesExistInconsistentNSMappingFails() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerDisabledClientDisabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(true, true);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
+        assertEquals(0, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create only SYSTEM:CATALOG
+    // and the second connection has client-side namespace mapping enabled
+    // Expected: Throw Inconsistent namespace mapping exception when you check client-server compatibility
+    @Test
+    public void testMappedSysCatExistsInconsistentNSMappingFails() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerDisabledClientEnabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(true, true);
+        try{
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertTrue(hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create all SYSTEM\..*
+    // tables and the second connection has client-side namespace mapping disabled
+    // Expected: All SYSTEM\..* tables exist and no upgrade is required
+    @Test
+    public void testNSMappingDisabledNoUpgradeRequired() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerDisabledClientDisabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(false, false);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
+        assertEquals(0, countUpgradeAttempts);
+    }
+
+    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create only SYSTEM:CATALOG
+    // and the second connection has client-side namespace mapping disabled
+    // Expected: The second connection should fail with Inconsistent namespace mapping exception
+    @Test
+    public void testClientNSMappingDisabledConnectionFails() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          firstConnectionNSMappingServerDisabledClientEnabled();
+        driver.resetCQS();
+        Properties clientProps = getClientProperties(false, false);
+        try{
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertTrue(hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, countUpgradeAttempts);
+    }
+
+    // Conditions: The first connection creates all SYSTEM tables via "EXECUTE UPGRADE" since auto-upgrade is disabled
+    // and the same client alters HBase metadata for SYSTEM.CATALOG
+    // Expected: Another client connection (with a new ConnectionQueryServices instance) made to the server does not
+    // revert the metadata change
+    @Test
+    public void testMetadataAlterRemainsAutoUpgradeDisabled() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver = firstConnectionAutoUpgradeToggle(false);
+        assertEquals(Integer.parseInt(MODIFIED_MAX_VERSIONS), verifyModificationTableMetadata(driver, PHOENIX_SYSTEM_CATALOG));
+    }
+
+    // Conditions: The first connection creates all SYSTEM tables (auto-upgrade is enabled) and the same client alters
+    // HBase metadata for SYSTEM.CATALOG
+    // Expected: Another client connection (with a new ConnectionQueryServices instance) made to the server does not
+    // revert the metadata change
+    @Test
+    public void testMetadataAlterRemainsAutoUpgradeEnabled() throws Exception {
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver = firstConnectionAutoUpgradeToggle(true);
+        assertEquals(Integer.parseInt(MODIFIED_MAX_VERSIONS), verifyModificationTableMetadata(driver, PHOENIX_SYSTEM_CATALOG));
+    }
+
+    /**
+     * Return all created HBase tables
+     * @return Set of HBase table name strings
+     * @throws IOException
+     */
+    private Set<String> getHBaseTables() throws IOException {
+        Set<String> tables = new HashSet<>();
+        for (TableName tn : testUtil.getHBaseAdmin().listTableNames()) {
+            tables.add(tn.getNameAsString());
+        }
+        return tables;
+    }
+
+    /**
+     * Alter the table metadata and return modified value
+     * @param driver
+     * @param tableName
+     * @return value of VERSIONS option for the table
+     * @throws Exception
+     */
+    private int verifyModificationTableMetadata(PhoenixSysCatCreationTestingDriver driver, String tableName) throws Exception {
+        // Modify table metadata
+        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), new Properties()).connect(getJdbcUrl(), new Properties());
+        conn.createStatement().execute("ALTER TABLE " + tableName + " SET VERSIONS = " + MODIFIED_MAX_VERSIONS);
+
+        // Connect via a client that creates a new ConnectionQueryServices instance
+        driver.resetCQS();
+        driver.getConnectionQueryServices(getJdbcUrl(), new Properties()).connect(getJdbcUrl(), new Properties());
+        HTableDescriptor descriptor = testUtil.getHBaseAdmin().getTableDescriptor(TableName.valueOf(tableName));
+        return descriptor.getFamily(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES).getMaxVersions();
+    }
+
+    /**
+     * Start the mini-cluster with server-side namespace mapping property specified
+     * @param isNamespaceMappingEnabled
+     * @throws Exception
+     */
+    private void startMiniClusterWithToggleNamespaceMapping(String isNamespaceMappingEnabled) throws Exception {
+        testUtil = new HBaseTestingUtility();
+        Configuration conf = testUtil.getConfiguration();
+        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, isNamespaceMappingEnabled);
+        // Avoid multiple clusters trying to bind to the master's info port (16010)
+        conf.setInt(HConstants.MASTER_INFO_PORT, -1);
+        testUtil.startMiniCluster(1);
+    }
+
+    /**
+     * Get the connection string for the mini-cluster
+     * @return Phoenix connection string
+     */
+    private String getJdbcUrl() {
+        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
+    }
+
+    /**
+     * Set namespace mapping related properties for the client connection
+     * @param nsMappingEnabled
+     * @param systemTableMappingEnabled
+     * @return Properties object
+     */
+    private Properties getClientProperties(boolean nsMappingEnabled, boolean systemTableMappingEnabled) {
+        Properties clientProps = new Properties();
+        clientProps.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.valueOf(nsMappingEnabled).toString());
+        clientProps.setProperty(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, Boolean.valueOf(systemTableMappingEnabled).toString());
+        return clientProps;
+    }
+
+    /**
+     * Initiate the first connection to the server with provided auto-upgrade property
+     * @param isAutoUpgradeEnabled
+     * @return Phoenix JDBC driver
+     * @throws Exception
+     */
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionAutoUpgradeToggle(boolean isAutoUpgradeEnabled)
+    throws Exception {
+        if (isAutoUpgradeEnabled) {
+            return firstConnectionNSMappingServerDisabledClientDisabled();
+        }
+        return firstConnectionAutoUpgradeDisabled();
+    }
+
+    // Conditions: isAutoUpgradeEnabled is false
+    // Expected: We do not create SYSTEM.CATALOG even if this is the first connection to the server. Later, when we manually
+    // run "EXECUTE UPGRADE", we create SYSTEM tables
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionAutoUpgradeDisabled() throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.FALSE.toString());
+        Map<String, String> props = new HashMap<>();
+        // Note that the isAutoUpgradeEnabled property is set when instantiating connection query services, not during init
+        props.put(QueryServices.AUTO_UPGRADE_ENABLED, Boolean.FALSE.toString());
+        ReadOnlyProps readOnlyProps = new ReadOnlyProps(props);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(readOnlyProps);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), new Properties());
+            fail("Client should not be able to create SYSTEM.CATALOG since we set the isAutoUpgradeEnabled property to false");
+        } catch (Exception e) {
+            assertTrue(e instanceof UpgradeRequiredException);
+        }
+        hbaseTables = getHBaseTables();
+        assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG) || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 0);
+        assertEquals(1, countUpgradeAttempts);
+
+        // We use the same ConnectionQueryServices instance to run "EXECUTE UPGRADE"
+        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), new Properties()).connect(getJdbcUrl(), new Properties());
+        try {
+            conn.createStatement().execute(EXECUTE_UPGRADE_COMMAND);
+        } finally {
+            conn.close();
+        }
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
+        return driver;
+    }
+
+    // Conditions: server-side namespace mapping is enabled, client-side namespace mapping is enabled and system tables
+    // are to be mapped to the SYSTEM namespace.
+    // Expected: If this is the first connection to the server, we should be able to create all namespace mapped system tables i.e. SYSTEM:.*
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerEnabledClientEnabled()
+    throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.TRUE.toString());
+        Properties clientProps = getClientProperties(true, true);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
+        assertEquals(0, countUpgradeAttempts);
+        return driver;
+    }
+
+    // Conditions: server-side namespace mapping is enabled, client-side namespace mapping is enabled, but mapping
+    // SYSTEM tables to the SYSTEM namespace is disabled
+    // Expected: If this is the first connection to the server, we will create unmapped SYSTEM tables i.e. SYSTEM\..*
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerEnabledClientEnabledMappingDisabled()
+    throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.TRUE.toString());
+        // client-side namespace mapping is enabled, but mapping SYSTEM tables to SYSTEM namespace is not
+        Properties clientProps = getClientProperties(true, false);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
+        assertEquals(0, countUpgradeAttempts);
+        return driver;
+    }
+
+    // Conditions: server-side namespace mapping is enabled, client-side namespace mapping is disabled
+    // Expected: Since this is the first connection to the server, we will create SYSTEM.CATALOG but immediately
+    // throw an exception for inconsistent namespace mapping
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerEnabledClientDisabled()
+    throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.TRUE.toString());
+        Properties clientProps = getClientProperties(false, false);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertTrue(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, countUpgradeAttempts);
+        return driver;
+    }
+
+    // Conditions: server-side namespace mapping is disabled, client-side namespace mapping is enabled
+    // Expected: Since this is the first connection to the server, we will create the SYSTEM namespace and create
+    // SYSTEM:CATALOG and then immediately throw an exception for inconsistent namespace mapping
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerDisabledClientEnabled()
+    throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.FALSE.toString());
+        Properties clientProps = getClientProperties(true, true);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
+        try {
+            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
+        } catch (SQLException sqlE) {
+            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
+        }
+        hbaseTables = getHBaseTables();
+        assertTrue(hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
+        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, countUpgradeAttempts);
+        return driver;
+    }
+
+    // Conditions: server-side namespace mapping is disabled, client-side namespace mapping is disabled
+    // Expected: Since this is the first connection to the server and auto-upgrade is enabled by default,
+    // we will create all SYSTEM\..* tables
+    private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerDisabledClientDisabled()
+    throws Exception {
+        startMiniClusterWithToggleNamespaceMapping(Boolean.FALSE.toString());
+        Properties clientProps = getClientProperties(false, false);
+        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
+          new SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver(ReadOnlyProps.EMPTY_PROPS);
+        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
+        hbaseTables = getHBaseTables();
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
+        assertEquals(0, countUpgradeAttempts);
+        return driver;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 a2d008b..34218d5 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
@@ -3615,6 +3615,27 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         long version = MetaDataUtil.encodeVersion(env.getHBaseVersion(), config);
 
+        PTable systemCatalog = null;
+        byte[] tableKey =
+          SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
+            PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA_BYTES,
+            PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE_BYTES);
+        ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
+        try {
+            systemCatalog = loadTable(env, tableKey, cacheKey, MIN_SYSTEM_TABLE_TIMESTAMP,
+              HConstants.LATEST_TIMESTAMP, request.getClientVersion());
+        } catch (Throwable t) {
+            logger.error("loading system catalog table inside getVersion failed", t);
+            ProtobufUtil.setControllerException(controller,
+              ServerUtil.createIOException(
+                SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
+                  isTablesMappingEnabled).toString(), t));
+        }
+        // In case this is the first connection, system catalog does not exist, and so we don't
+        // set the optional system catalog timestamp.
+        if (systemCatalog != null) {
+            builder.setSystemCatalogTimestamp(systemCatalog.getTimeStamp());
+        }
         builder.setVersion(version);
         done.run(builder.build());
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 a71ce0c..26f8198 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
@@ -448,6 +448,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
         return iterator.next();
     }
     
+    public static long getPriorUpgradeVersion() {
+        return TIMESTAMP_VERSION_MAP.lowerKey(TIMESTAMP_VERSION_MAP.lastKey());
+    }
+
     public static String getVersion(long serverTimestamp) {
         /*
          * It is possible that when clients are trying to run upgrades concurrently, we could be at an intermediate

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index e041abd..8119c6e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@ -14117,6 +14117,16 @@ public final class MetaDataProtos {
      * <code>required int64 version = 1;</code>
      */
     long getVersion();
+
+    // optional int64 systemCatalogTimestamp = 2;
+    /**
+     * <code>optional int64 systemCatalogTimestamp = 2;</code>
+     */
+    boolean hasSystemCatalogTimestamp();
+    /**
+     * <code>optional int64 systemCatalogTimestamp = 2;</code>
+     */
+    long getSystemCatalogTimestamp();
   }
   /**
    * Protobuf type {@code GetVersionResponse}
@@ -14174,6 +14184,11 @@ public final class MetaDataProtos {
               version_ = input.readInt64();
               break;
             }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              systemCatalogTimestamp_ = input.readInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -14230,8 +14245,25 @@ public final class MetaDataProtos {
       return version_;
     }
 
+    // optional int64 systemCatalogTimestamp = 2;
+    public static final int SYSTEMCATALOGTIMESTAMP_FIELD_NUMBER = 2;
+    private long systemCatalogTimestamp_;
+    /**
+     * <code>optional int64 systemCatalogTimestamp = 2;</code>
+     */
+    public boolean hasSystemCatalogTimestamp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional int64 systemCatalogTimestamp = 2;</code>
+     */
+    public long getSystemCatalogTimestamp() {
+      return systemCatalogTimestamp_;
+    }
+
     private void initFields() {
       version_ = 0L;
+      systemCatalogTimestamp_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -14252,6 +14284,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt64(1, version_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt64(2, systemCatalogTimestamp_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -14265,6 +14300,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(1, version_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(2, systemCatalogTimestamp_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -14293,6 +14332,11 @@ public final class MetaDataProtos {
         result = result && (getVersion()
             == other.getVersion());
       }
+      result = result && (hasSystemCatalogTimestamp() == other.hasSystemCatalogTimestamp());
+      if (hasSystemCatalogTimestamp()) {
+        result = result && (getSystemCatalogTimestamp()
+            == other.getSystemCatalogTimestamp());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -14310,6 +14354,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + VERSION_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getVersion());
       }
+      if (hasSystemCatalogTimestamp()) {
+        hash = (37 * hash) + SYSTEMCATALOGTIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSystemCatalogTimestamp());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -14421,6 +14469,8 @@ public final class MetaDataProtos {
         super.clear();
         version_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000001);
+        systemCatalogTimestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -14453,6 +14503,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.version_ = version_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.systemCatalogTimestamp_ = systemCatalogTimestamp_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -14472,6 +14526,9 @@ public final class MetaDataProtos {
         if (other.hasVersion()) {
           setVersion(other.getVersion());
         }
+        if (other.hasSystemCatalogTimestamp()) {
+          setSystemCatalogTimestamp(other.getSystemCatalogTimestamp());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -14536,6 +14593,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int64 systemCatalogTimestamp = 2;
+      private long systemCatalogTimestamp_ ;
+      /**
+       * <code>optional int64 systemCatalogTimestamp = 2;</code>
+       */
+      public boolean hasSystemCatalogTimestamp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional int64 systemCatalogTimestamp = 2;</code>
+       */
+      public long getSystemCatalogTimestamp() {
+        return systemCatalogTimestamp_;
+      }
+      /**
+       * <code>optional int64 systemCatalogTimestamp = 2;</code>
+       */
+      public Builder setSystemCatalogTimestamp(long value) {
+        bitField0_ |= 0x00000002;
+        systemCatalogTimestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 systemCatalogTimestamp = 2;</code>
+       */
+      public Builder clearSystemCatalogTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        systemCatalogTimestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:GetVersionResponse)
     }
 
@@ -17103,51 +17193,52 @@ public final class MetaDataProtos {
       "cheRequest\022\025\n\rclientVersion\030\001 \001(\005\"*\n\022Cle" +
       "arCacheResponse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n" +
       "\021GetVersionRequest\022\025\n\rclientVersion\030\001 \001(" +
-      "\005\"%\n\022GetVersionResponse\022\017\n\007version\030\001 \002(\003" +
-      "\"\205\001\n\032ClearTableFromCacheRequest\022\020\n\010tenan",
-      "tId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableNa" +
-      "me\030\003 \002(\014\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rcli" +
-      "entVersion\030\005 \001(\005\"\035\n\033ClearTableFromCacheR" +
-      "esponse*\365\004\n\014MutationCode\022\030\n\024TABLE_ALREAD" +
-      "Y_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUM" +
-      "N_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003" +
-      "\022\035\n\031CONCURRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE" +
-      "_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022" +
-      "\034\n\030UNALLOWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_C" +
-      "OLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027",
-      "FUNCTION_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NO" +
-      "T_FOUND\020\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026F" +
-      "UNCTION_NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREAD" +
-      "Y_EXISTS\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SC" +
-      "HEMA_NOT_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION" +
-      "\020\021\022\032\n\026TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLO" +
-      "WED_SCHEMA_MUTATION\020\023\022%\n!AUTO_PARTITION_" +
-      "SEQUENCE_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AU" +
-      "TO_PARTITION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\0262" +
-      "\345\006\n\017MetaDataService\022/\n\010getTable\022\020.GetTab",
-      "leRequest\032\021.MetaDataResponse\0227\n\014getFunct" +
-      "ions\022\024.GetFunctionsRequest\032\021.MetaDataRes" +
-      "ponse\0221\n\tgetSchema\022\021.GetSchemaRequest\032\021." +
-      "MetaDataResponse\0225\n\013createTable\022\023.Create" +
-      "TableRequest\032\021.MetaDataResponse\022;\n\016creat" +
-      "eFunction\022\026.CreateFunctionRequest\032\021.Meta" +
-      "DataResponse\0227\n\014createSchema\022\024.CreateSch" +
-      "emaRequest\032\021.MetaDataResponse\0221\n\tdropTab" +
-      "le\022\021.DropTableRequest\032\021.MetaDataResponse" +
-      "\0223\n\ndropSchema\022\022.DropSchemaRequest\032\021.Met",
-      "aDataResponse\0227\n\014dropFunction\022\024.DropFunc" +
-      "tionRequest\032\021.MetaDataResponse\0221\n\taddCol" +
-      "umn\022\021.AddColumnRequest\032\021.MetaDataRespons" +
-      "e\0223\n\ndropColumn\022\022.DropColumnRequest\032\021.Me" +
-      "taDataResponse\022?\n\020updateIndexState\022\030.Upd" +
-      "ateIndexStateRequest\032\021.MetaDataResponse\022" +
-      "5\n\nclearCache\022\022.ClearCacheRequest\032\023.Clea" +
-      "rCacheResponse\0225\n\ngetVersion\022\022.GetVersio" +
-      "nRequest\032\023.GetVersionResponse\022P\n\023clearTa" +
-      "bleFromCache\022\033.ClearTableFromCacheReques",
-      "t\032\034.ClearTableFromCacheResponseBB\n(org.a" +
-      "pache.phoenix.coprocessor.generatedB\016Met" +
-      "aDataProtosH\001\210\001\001\240\001\001"
+      "\005\"E\n\022GetVersionResponse\022\017\n\007version\030\001 \002(\003" +
+      "\022\036\n\026systemCatalogTimestamp\030\002 \001(\003\"\205\001\n\032Cle",
+      "arTableFromCacheRequest\022\020\n\010tenantId\030\001 \002(" +
+      "\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014" +
+      "\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rclientVersi" +
+      "on\030\005 \001(\005\"\035\n\033ClearTableFromCacheResponse*" +
+      "\365\004\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXISTS" +
+      "\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FO" +
+      "UND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONC" +
+      "URRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_" +
+      "REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALL" +
+      "OWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010",
+      "\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION" +
+      "_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020" +
+      "\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026FUNCTION_" +
+      "NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREADY_EXISTS" +
+      "\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SCHEMA_NOT" +
+      "_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION\020\021\022\032\n\026TA" +
+      "BLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLOWED_SCHE" +
+      "MA_MUTATION\020\023\022%\n!AUTO_PARTITION_SEQUENCE" +
+      "_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AUTO_PARTI" +
+      "TION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\0262\345\006\n\017Meta",
+      "DataService\022/\n\010getTable\022\020.GetTableReques" +
+      "t\032\021.MetaDataResponse\0227\n\014getFunctions\022\024.G" +
+      "etFunctionsRequest\032\021.MetaDataResponse\0221\n" +
+      "\tgetSchema\022\021.GetSchemaRequest\032\021.MetaData" +
+      "Response\0225\n\013createTable\022\023.CreateTableReq" +
+      "uest\032\021.MetaDataResponse\022;\n\016createFunctio" +
+      "n\022\026.CreateFunctionRequest\032\021.MetaDataResp" +
+      "onse\0227\n\014createSchema\022\024.CreateSchemaReque" +
+      "st\032\021.MetaDataResponse\0221\n\tdropTable\022\021.Dro" +
+      "pTableRequest\032\021.MetaDataResponse\0223\n\ndrop",
+      "Schema\022\022.DropSchemaRequest\032\021.MetaDataRes" +
+      "ponse\0227\n\014dropFunction\022\024.DropFunctionRequ" +
+      "est\032\021.MetaDataResponse\0221\n\taddColumn\022\021.Ad" +
+      "dColumnRequest\032\021.MetaDataResponse\0223\n\ndro" +
+      "pColumn\022\022.DropColumnRequest\032\021.MetaDataRe" +
+      "sponse\022?\n\020updateIndexState\022\030.UpdateIndex" +
+      "StateRequest\032\021.MetaDataResponse\0225\n\nclear" +
+      "Cache\022\022.ClearCacheRequest\032\023.ClearCacheRe" +
+      "sponse\0225\n\ngetVersion\022\022.GetVersionRequest" +
+      "\032\023.GetVersionResponse\022P\n\023clearTableFromC",
+      "ache\022\033.ClearTableFromCacheRequest\032\034.Clea" +
+      "rTableFromCacheResponseBB\n(org.apache.ph" +
+      "oenix.coprocessor.generatedB\016MetaDataPro" +
+      "tosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17261,7 +17352,7 @@ public final class MetaDataProtos {
           internal_static_GetVersionResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetVersionResponse_descriptor,
-              new java.lang.String[] { "Version", });
+              new java.lang.String[] { "Version", "SystemCatalogTimestamp", });
           internal_static_ClearTableFromCacheRequest_descriptor =
             getDescriptor().getMessageTypes().get(18);
           internal_static_ClearTableFromCacheRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-core/src/main/java/org/apache/phoenix/exception/UpgradeRequiredException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/UpgradeRequiredException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/UpgradeRequiredException.java
index 9352a50..6c9706b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/UpgradeRequiredException.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/UpgradeRequiredException.java
@@ -17,12 +17,23 @@
  */
 package org.apache.phoenix.exception;
 
+import org.apache.hadoop.hbase.HConstants;
 
 public class UpgradeRequiredException extends RetriableUpgradeException {
+    private final long systemCatalogTimestamp;
 
     public UpgradeRequiredException() {
+        this(HConstants.OLDEST_TIMESTAMP);
+    }
+
+    public UpgradeRequiredException(long systemCatalogTimeStamp) {
         super("Operation not allowed since cluster hasn't been upgraded. Call EXECUTE UPGRADE. ",
-                SQLExceptionCode.UPGRADE_REQUIRED.getSQLState(), SQLExceptionCode.UPGRADE_REQUIRED.getErrorCode());
+          SQLExceptionCode.UPGRADE_REQUIRED.getSQLState(), SQLExceptionCode.UPGRADE_REQUIRED.getErrorCode());
+        this.systemCatalogTimestamp = systemCatalogTimeStamp;
+    }
+
+    public long getSystemCatalogTimeStamp() {
+        return systemCatalogTimestamp;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 b75119b..72bfa83 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
@@ -84,7 +84,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
 
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] tableName, PTableType tableType,
             Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException;
+            boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet) throws SQLException;
     public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException;
     public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists) throws SQLException;
     public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException;


[04/21] phoenix git commit: PHOENIX-2715 Query Log (Ankit Singhal)

Posted by pb...@apache.org.
PHOENIX-2715 Query Log (Ankit Singhal)


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

Branch: refs/heads/4.x-cdh5.12
Commit: b291068bc9c6e133c7bcb6dfe52dd4f96a76f2a1
Parents: 4d9cc92
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Apr 10 07:53:31 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:26:16 2018 +0100

----------------------------------------------------------------------
 phoenix-core/pom.xml                            |   5 +
 .../end2end/QueryDatabaseMetaDataIT.java        |   4 +
 .../apache/phoenix/end2end/QueryLoggerIT.java   | 358 +++++++++++++++++++
 .../end2end/TenantSpecificTablesDDLIT.java      |   2 +
 .../phoenix/compile/StatementContext.java       |  10 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   5 +-
 .../phoenix/iterate/ScanningResultIterator.java |  18 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  17 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  18 +
 .../phoenix/jdbc/PhoenixPreparedStatement.java  |  11 +-
 .../apache/phoenix/jdbc/PhoenixResultSet.java   |  38 ++
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  67 +++-
 .../java/org/apache/phoenix/log/LogLevel.java   |  22 ++
 .../java/org/apache/phoenix/log/LogWriter.java  |  51 +++
 .../log/QueryLogDetailsEventHandler.java        |  63 ++++
 .../org/apache/phoenix/log/QueryLogInfo.java    |  87 +++++
 .../org/apache/phoenix/log/QueryLogState.java   |  22 ++
 .../org/apache/phoenix/log/QueryLogger.java     | 145 ++++++++
 .../log/QueryLoggerDefaultExceptionHandler.java |  51 +++
 .../phoenix/log/QueryLoggerDisruptor.java       | 117 ++++++
 .../org/apache/phoenix/log/QueryLoggerUtil.java |  62 ++++
 .../org/apache/phoenix/log/RingBufferEvent.java |  93 +++++
 .../phoenix/log/RingBufferEventTranslator.java  |  53 +++
 .../org/apache/phoenix/log/TableLogWriter.java  | 125 +++++++
 .../phoenix/monitoring/ReadMetricQueue.java     |  44 ++-
 .../phoenix/monitoring/ScanMetricsHolder.java   |  48 ++-
 .../phoenix/query/ConnectionQueryServices.java  |   6 +
 .../query/ConnectionQueryServicesImpl.java      |  35 +-
 .../query/ConnectionlessQueryServicesImpl.java  |  18 +
 .../query/DelegateConnectionQueryServices.java  |  14 +
 .../apache/phoenix/query/QueryConstants.java    |  45 +++
 .../org/apache/phoenix/query/QueryServices.java |   4 +
 .../phoenix/query/QueryServicesOptions.java     |   9 +-
 pom.xml                                         |   6 +
 34 files changed, 1612 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index b07cbbb..8fe8a10 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -526,5 +526,10 @@
       <artifactId>i18n-util</artifactId>
       <version>${i18n-util.version}</version>
     </dependency>
+       <dependency>
+        <groupId>com.lmax</groupId>
+        <artifactId>disruptor</artifactId>
+        <version>${disruptor.version}</version>
+      </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index a1bcf40..54cb5da 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -166,6 +166,10 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
             assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
             assertTrue(rs.next());
             assertEquals(SYSTEM_CATALOG_SCHEMA, rs.getString("TABLE_SCHEM"));
+            assertEquals(PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE, rs.getString("TABLE_NAME"));
+            assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
+            assertTrue(rs.next());
+            assertEquals(SYSTEM_CATALOG_SCHEMA, rs.getString("TABLE_SCHEM"));
             assertEquals(TYPE_SEQUENCE, rs.getString("TABLE_NAME"));
             assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryLoggerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryLoggerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryLoggerIT.java
new file mode 100644
index 0000000..940ba6f
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryLoggerIT.java
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+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.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TOTAL_EXECUTION_TIME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.net.InetAddress;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
+import org.apache.phoenix.log.LogLevel;
+import org.apache.phoenix.log.QueryLogState;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class QueryLoggerIT extends BaseUniqueNamesOwnClusterIT {
+
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        // Enable request metric collection at the driver level
+        props.put(QueryServices.COLLECT_REQUEST_LEVEL_METRICS, String.valueOf(true));
+        // disable renewing leases as this will force spooling to happen.
+        props.put(QueryServices.RENEW_LEASE_ENABLED, String.valueOf(false));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        // need the non-test driver for some tests that check number of hconnections, etc.
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    } 
+    
+
+    @Test
+    public void testDebugLogs() throws Exception {
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
+        Properties props= new Properties();
+        props.setProperty(QueryServices.LOG_LEVEL, LogLevel.DEBUG.name());
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        assertEquals(conn.unwrap(PhoenixConnection.class).getLogLevel(),LogLevel.DEBUG);
+        String query = "SELECT * FROM " + tableName;
+        ResultSet rs = conn.createStatement().executeQuery(query);
+        StatementContext context = ((PhoenixResultSet)rs).getContext();
+        String queryId = context.getQueryLogger().getQueryId();
+        while (rs.next()) {
+            rs.getString(1);
+            rs.getString(2);
+        }
+        ResultSet explainRS = conn.createStatement().executeQuery("Explain " + query);
+
+        String logQuery = "SELECT * FROM " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"";
+        rs = conn.createStatement().executeQuery(logQuery);
+        boolean foundQueryLog = false;
+        int delay = 5000;
+
+        // sleep for sometime to let query log committed
+        Thread.sleep(delay);
+        while (rs.next()) {
+            if (rs.getString(QUERY_ID).equals(queryId)) {
+                foundQueryLog = true;
+                assertEquals(rs.getString(BIND_PARAMETERS), null);
+                assertEquals(rs.getString(USER), System.getProperty("user.name"));
+                assertEquals(rs.getString(CLIENT_IP), InetAddress.getLocalHost().getHostAddress());
+                assertEquals(rs.getString(EXPLAIN_PLAN), QueryUtil.getExplainPlan(explainRS));
+                assertEquals(rs.getString(GLOBAL_SCAN_DETAILS), context.getScan().toJSON());
+                assertEquals(rs.getLong(NO_OF_RESULTS_ITERATED), 10);
+                assertEquals(rs.getString(QUERY), query);
+                assertEquals(rs.getString(QUERY_STATUS), QueryLogState.COMPLETED.toString());
+                assertTrue(System.currentTimeMillis() - rs.getTimestamp(START_TIME).getTime() > delay);
+                assertEquals(rs.getString(TENANT_ID), null);
+                assertTrue(rs.getString(TOTAL_EXECUTION_TIME) != null);
+                assertTrue(rs.getString(SCAN_METRICS_JSON).contains("scanMetrics"));
+                assertEquals(rs.getString(EXCEPTION_TRACE),null);
+            }else{
+                //confirm we are not logging system queries
+                assertFalse(rs.getString(QUERY).toString().contains(SYSTEM_CATALOG_SCHEMA));
+            }
+        }
+        assertTrue(foundQueryLog);
+        conn.close();
+    }
+    
+    @Test
+    public void testLogSampling() throws Exception {
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
+        Properties props= new Properties();
+        props.setProperty(QueryServices.LOG_LEVEL, LogLevel.DEBUG.name());
+        props.setProperty(QueryServices.LOG_SAMPLE_RATE, "0.5");
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        assertEquals(conn.unwrap(PhoenixConnection.class).getLogLevel(),LogLevel.DEBUG);
+        String query = "SELECT * FROM " + tableName;
+        int count=100;
+        for (int i = 0; i < count; i++) {
+            conn.createStatement().executeQuery(query);
+        }
+        
+        String logQuery = "SELECT * FROM " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"";
+        ResultSet rs = conn.createStatement().executeQuery(logQuery);
+        int delay = 5000;
+
+        // sleep for sometime to let query log committed
+        Thread.sleep(delay);
+        int logCount=0;
+        while (rs.next()) {
+            logCount++;
+        }
+        
+        //sampling rate is 0.5 , but with lesser count, uniformity of thread random may not be perfect, so taking 0.75 for comparison 
+        assertTrue(logCount != 0 && logCount < count * 0.75);
+        conn.close();
+    }
+    
+    @Test
+    public void testInfoLogs() throws Exception{
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
+        Properties props= new Properties();
+        props.setProperty(QueryServices.LOG_LEVEL, LogLevel.INFO.name());
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        assertEquals(conn.unwrap(PhoenixConnection.class).getLogLevel(),LogLevel.INFO);
+        String query = "SELECT * FROM " + tableName;
+        
+        ResultSet rs = conn.createStatement().executeQuery(query);
+        StatementContext context = ((PhoenixResultSet)rs).getContext();
+        String queryId = context.getQueryLogger().getQueryId();
+        while (rs.next()) {
+            rs.getString(1);
+            rs.getString(2);
+        }
+
+        String logQuery = "SELECT * FROM " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"";
+        rs = conn.createStatement().executeQuery(logQuery);
+        boolean foundQueryLog = false;
+        int delay = 5000;
+
+        // sleep for sometime to let query log committed
+        Thread.sleep(delay);
+        while (rs.next()) {
+            if (rs.getString(QUERY_ID).equals(queryId)) {
+                foundQueryLog = true;
+                assertEquals(rs.getString(USER), System.getProperty("user.name"));
+                assertEquals(rs.getString(CLIENT_IP), InetAddress.getLocalHost().getHostAddress());
+                assertEquals(rs.getString(EXPLAIN_PLAN), null);
+                assertEquals(rs.getString(GLOBAL_SCAN_DETAILS),null);
+                assertEquals(rs.getLong(NO_OF_RESULTS_ITERATED), 0);
+                assertEquals(rs.getString(QUERY), query);
+                assertEquals(rs.getString(QUERY_STATUS),null);
+                assertTrue(System.currentTimeMillis() - rs.getTimestamp(START_TIME).getTime() > delay);
+                assertEquals(rs.getString(TENANT_ID), null);
+                assertTrue(rs.getString(TOTAL_EXECUTION_TIME) == null);
+            }
+        }
+        assertTrue(foundQueryLog);
+        conn.close();
+    }
+    
+    @Test
+    public void testWithLoggingOFF() throws Exception{
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
+        Properties props= new Properties();
+        props.setProperty(QueryServices.LOG_LEVEL, LogLevel.OFF.name());
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        assertEquals(conn.unwrap(PhoenixConnection.class).getLogLevel(),LogLevel.OFF);
+        String query = "SELECT * FROM " + tableName;
+        
+        ResultSet rs = conn.createStatement().executeQuery(query);
+        StatementContext context = ((PhoenixResultSet)rs).getContext();
+        String queryId = context.getQueryLogger().getQueryId();
+        while (rs.next()) {
+            rs.getString(1);
+            rs.getString(2);
+        }
+
+        String logQuery = "SELECT * FROM " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"";
+        rs = conn.createStatement().executeQuery(logQuery);
+        boolean foundQueryLog = false;
+        int delay = 5000;
+
+        // sleep for sometime to let query log committed
+        Thread.sleep(delay);
+        while (rs.next()) {
+            if (rs.getString(QUERY_ID).equals(queryId)) {
+                foundQueryLog = true;
+            }
+        }
+        assertFalse(foundQueryLog);
+        conn.close();
+    }
+    
+
+    @Test
+    public void testPreparedStatementWithTrace() throws Exception{
+        testPreparedStatement(LogLevel.TRACE);   
+    }
+    
+    @Test
+    public void testPreparedStatementWithDebug() throws Exception{
+        testPreparedStatement(LogLevel.DEBUG);
+    }
+            
+    private void testPreparedStatement(LogLevel loglevel) throws Exception{
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
+        Properties props= new Properties();
+        props.setProperty(QueryServices.LOG_LEVEL, loglevel.name());
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        assertEquals(conn.unwrap(PhoenixConnection.class).getLogLevel(),loglevel);
+        
+        String query = "SELECT * FROM " + tableName +" where V = ?";
+        
+        PreparedStatement pstmt = conn.prepareStatement(query);
+        pstmt.setString(1, "value5");
+        ResultSet rs = pstmt.executeQuery();
+        StatementContext context = ((PhoenixResultSet)rs).getContext();
+        String queryId = context.getQueryLogger().getQueryId();
+        while (rs.next()) {
+            rs.getString(1);
+            rs.getString(2);
+        }
+        ResultSet explainRS = conn.createStatement()
+                .executeQuery("Explain " + "SELECT * FROM " + tableName + " where V = 'value5'");
+        String logQuery = "SELECT * FROM " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"";
+        rs = conn.createStatement().executeQuery(logQuery);
+        boolean foundQueryLog = false;
+        int delay = 5000;
+        
+        // sleep for sometime to let query log committed
+        Thread.sleep(delay);
+        while (rs.next()) {
+            if (rs.getString(QUERY_ID).equals(queryId)) {
+                foundQueryLog = true;
+                assertEquals(rs.getString(BIND_PARAMETERS), loglevel == LogLevel.TRACE ? "value5" : null);
+                assertEquals(rs.getString(USER), System.getProperty("user.name"));
+                assertEquals(rs.getString(CLIENT_IP), InetAddress.getLocalHost().getHostAddress());
+                assertEquals(rs.getString(EXPLAIN_PLAN), QueryUtil.getExplainPlan(explainRS));
+                assertEquals(rs.getString(GLOBAL_SCAN_DETAILS), context.getScan().toJSON());
+                assertEquals(rs.getLong(NO_OF_RESULTS_ITERATED), 1);
+                assertEquals(rs.getString(QUERY), query);
+                assertEquals(rs.getString(QUERY_STATUS), QueryLogState.COMPLETED.toString());
+                assertTrue(System.currentTimeMillis() - rs.getTimestamp(START_TIME).getTime() > delay);
+                assertEquals(rs.getString(TENANT_ID), null);
+                assertTrue(rs.getString(TOTAL_EXECUTION_TIME) != null);
+            }
+        }
+        assertTrue(foundQueryLog);
+        conn.close();
+    }
+    
+    
+    
+    @Test
+    public void testFailedQuery() throws Exception {
+        String tableName = generateUniqueName();
+        Properties props = new Properties();
+        props.setProperty(QueryServices.LOG_LEVEL, LogLevel.DEBUG.name());
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        assertEquals(conn.unwrap(PhoenixConnection.class).getLogLevel(), LogLevel.DEBUG);
+        // Table does not exists
+        String query = "SELECT * FROM " + tableName;
+
+        try {
+            conn.createStatement().executeQuery(query);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(e.getErrorCode(), SQLExceptionCode.TABLE_UNDEFINED.getErrorCode());
+        }
+        String logQuery = "SELECT * FROM " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"";
+        ResultSet rs = conn.createStatement().executeQuery(logQuery);
+        boolean foundQueryLog = false;
+        int delay = 5000;
+
+        // sleep for sometime to let query log committed
+        Thread.sleep(delay);
+        while (rs.next()) {
+            if (QueryLogState.FAILED.name().equals(rs.getString(QUERY_STATUS))) {
+                foundQueryLog = true;
+                assertEquals(rs.getString(USER), System.getProperty("user.name"));
+                assertEquals(rs.getString(CLIENT_IP), InetAddress.getLocalHost().getHostAddress());
+                assertEquals(rs.getString(EXPLAIN_PLAN), null);
+                assertEquals(rs.getString(GLOBAL_SCAN_DETAILS), null);
+                assertEquals(rs.getLong(NO_OF_RESULTS_ITERATED), 0);
+                assertEquals(rs.getString(QUERY), query);
+                assertTrue(rs.getString(EXCEPTION_TRACE).contains(SQLExceptionCode.TABLE_UNDEFINED.getMessage()));
+                assertTrue(System.currentTimeMillis() - rs.getTimestamp(START_TIME).getTime() > delay);
+                assertTrue(rs.getString(TOTAL_EXECUTION_TIME) != null);
+            }
+        }
+        assertTrue(foundQueryLog);
+        conn.close();
+    }
+
+    private static void createTableAndInsertValues(String tableName, boolean resetGlobalMetricsAfterTableCreate)
+            throws Exception {
+        String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(ddl);
+        // executing 10 upserts/mutations.
+        String dml = "UPSERT INTO " + tableName + " VALUES (?, ?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        for (int i = 1; i <= 10; i++) {
+            stmt.setString(1, "key" + i);
+            stmt.setString(2, "value" + i);
+            stmt.executeUpdate();
+        }
+        conn.commit();
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index f8dfd65..34a1312 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -493,6 +493,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, SYSTEM_FUNCTION_TABLE, SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE, PTableType.SYSTEM);
+            assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);
             assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, PTableType.SYSTEM);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
index 39d8525..c105046 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.log.QueryLogger;
 import org.apache.phoenix.monitoring.OverAllQueryMetrics;
 import org.apache.phoenix.monitoring.ReadMetricQueue;
 import org.apache.phoenix.parse.SelectStatement;
@@ -83,6 +84,7 @@ public class StatementContext {
     private Map<SelectStatement, Object> subqueryResults;
     private final ReadMetricQueue readMetricsQueue;
     private final OverAllQueryMetrics overAllQueryMetrics;
+    private QueryLogger queryLogger;
     
     public StatementContext(PhoenixStatement statement) {
         this(statement, new Scan());
@@ -306,5 +308,13 @@ public class StatementContext {
     public OverAllQueryMetrics getOverallQueryMetrics() {
         return overAllQueryMetrics;
     }
+
+    public void setQueryLogger(QueryLogger queryLogger) {
+       this.queryLogger=queryLogger;
+    }
+
+    public QueryLogger getQueryLogger() {
+        return queryLogger;
+    }
     
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 efad1e7..4c4c96f 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
@@ -76,6 +76,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
     public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
     public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
+    public static final int DEFAULT_LOG_VERSIONS = 10;
+    public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
     
     // Min system table timestamps for every release.
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 = MIN_TABLE_TIMESTAMP + 3;
@@ -118,7 +120,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0, "4.14.x");
     }
     
-    public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION + "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER; 
+    public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION + "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER;
+     
     
     // TODO: pare this down to minimum, as we don't need duplicates for both table and column errors, nor should we need
     // a different code for every type of error.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
index 011feaa..9a31238 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
@@ -17,18 +17,18 @@
  */
 package org.apache.phoenix.iterate;
 
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_SCAN_BYTES;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_RPC_CALLS;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_REMOTE_RPC_CALLS;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_BYTES_IN_REMOTE_RESULTS;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_BYTES_REGION_SERVER_RESULTS;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_MILLS_BETWEEN_NEXTS;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_NOT_SERVING_REGION_EXCEPTION;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_BYTES_REGION_SERVER_RESULTS;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_BYTES_IN_REMOTE_RESULTS;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_SCANNED_REGIONS;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_RPC_RETRIES;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_REMOTE_RPC_CALLS;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_REMOTE_RPC_RETRIES;
-import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_ROWS_SCANNED;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_ROWS_FILTERED;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_ROWS_SCANNED;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_RPC_CALLS;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_RPC_RETRIES;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_SCANNED_REGIONS;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_SCAN_BYTES;
 
 import java.io.IOException;
 import java.sql.SQLException;
@@ -93,7 +93,7 @@ public class ScanningResultIterator implements ResultIterator {
             if(scanMetricsMap == null) {
                 return;
             }
-
+            scanMetricsHolder.setScanMetricMap(scanMetricsMap);
             changeMetric(scanMetricsHolder.getCountOfRPCcalls(),
                     scanMetricsMap.get(RPC_CALLS_METRIC_NAME));
             changeMetric(scanMetricsHolder.getCountOfRemoteRPCcalls(),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 2b428c9..d3626f8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -75,11 +75,11 @@ import org.apache.phoenix.iterate.ParallelIteratorFactory;
 import org.apache.phoenix.iterate.TableResultIterator;
 import org.apache.phoenix.iterate.TableResultIteratorFactory;
 import org.apache.phoenix.jdbc.PhoenixStatement.PhoenixStatementParser;
+import org.apache.phoenix.log.LogLevel;
 import org.apache.phoenix.monitoring.MetricType;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.query.ConnectionQueryServices;
-import org.apache.phoenix.query.ConnectionQueryServices.Feature;
 import org.apache.phoenix.query.DelegateConnectionQueryServices;
 import org.apache.phoenix.query.MetaDataMutated;
 import org.apache.phoenix.query.PropertyPolicyProvider;
@@ -168,6 +168,8 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     private final LinkedBlockingQueue<WeakReference<TableResultIterator>> scannerQueue;
     private TableResultIteratorFactory tableResultIteratorFactory;
     private boolean isRunningUpgrade;
+    private LogLevel logLevel;
+    private Double logSamplingRate;
 
     static {
         Tracing.addTraceMetricsSource();
@@ -378,6 +380,10 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
         this.scannerQueue = new LinkedBlockingQueue<>();
         this.tableResultIteratorFactory = new DefaultTableResultIteratorFactory();
         this.isRunningUpgrade = isRunningUpgrade;
+        this.logLevel= LogLevel.valueOf(this.services.getProps().get(QueryServices.LOG_LEVEL,
+                QueryServicesOptions.DEFAULT_LOGGING_LEVEL));
+        this.logSamplingRate = Double.parseDouble(this.services.getProps().get(QueryServices.LOG_SAMPLE_RATE,
+                QueryServicesOptions.DEFAULT_LOG_SAMPLE_RATE));
         GLOBAL_OPEN_PHOENIX_CONNECTIONS.increment();
     }
 
@@ -648,6 +654,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
             } finally {
                 services.removeConnection(this);
             }
+            
         } finally {
             isClosed = true;
             GLOBAL_OPEN_PHOENIX_CONNECTIONS.decrement();
@@ -1274,4 +1281,12 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
         this.isRunningUpgrade = isRunningUpgrade;
     }
 
+    public LogLevel getLogLevel(){
+        return this.logLevel;
+    }
+    
+    public Double getLogSamplingRate(){
+        return this.logSamplingRate;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 b88b381..9caf7fb 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
@@ -343,6 +343,24 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String USE_STATS_FOR_PARALLELIZATION = "USE_STATS_FOR_PARALLELIZATION";
     public static final byte[] USE_STATS_FOR_PARALLELIZATION_BYTES = Bytes.toBytes(USE_STATS_FOR_PARALLELIZATION);
 
+    
+    //SYSTEM:LOG
+    public static final String SYSTEM_LOG_TABLE = "LOG";
+    public static final String QUERY_ID = "QUERY_ID";
+    public static final String USER = "USER";
+    public static final String CLIENT_IP = "CLIENT_IP";
+    public static final String QUERY = "QUERY";
+    public static final String EXPLAIN_PLAN = "EXPLAIN_PLAN";
+    public static final String TOTAL_EXECUTION_TIME = "TOTAL_EXECUTION_TIME";
+    public static final String NO_OF_RESULTS_ITERATED = "NO_OF_RESULTS_ITERATED";
+    public static final String QUERY_STATUS = "QUERY_STATUS";
+    public static final String EXCEPTION_TRACE = "EXCEPTION_TRACE";
+    public static final String GLOBAL_SCAN_DETAILS = "GLOBAL_SCAN_DETAILS";
+    public static final String SCAN_METRICS_JSON = "SCAN_METRICS_JSON";
+    public static final String START_TIME = "START_TIME";
+    public static final String BIND_PARAMETERS = "BIND_PARAMETERS";
+            
+    
     PhoenixDatabaseMetaData(PhoenixConnection connection) throws SQLException {
         this.emptyResultSet = new PhoenixResultSet(ResultIterator.EMPTY_ITERATOR, RowProjector.EMPTY_PROJECTOR, new StatementContext(new PhoenixStatement(connection), false));
         this.connection = connection;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
index 71ecb8d..914ea33 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
@@ -169,7 +169,13 @@ public class PhoenixPreparedStatement extends PhoenixStatement implements Prepar
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.EXECUTE_UPDATE_WITH_NON_EMPTY_BATCH)
             .build().buildException();
         }
-        return execute(statement);
+        if (statement.getOperation().isMutation()) {
+            executeMutation(statement);
+            return false;
+        }
+        executeQuery(statement, createQueryLogger(statement,query));
+        return true;
+        
     }
 
     @Override
@@ -183,7 +189,8 @@ public class PhoenixPreparedStatement extends PhoenixStatement implements Prepar
         if (statement.getOperation().isMutation()) {
             throw new ExecuteQueryNotApplicableException(statement.getOperation());
         }
-        return executeQuery(statement);
+        
+        return executeQuery(statement,createQueryLogger(statement,query));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
index d3ec151..153fa08 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
@@ -50,6 +50,9 @@ import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.log.QueryLogInfo;
+import org.apache.phoenix.log.QueryLogState;
+import org.apache.phoenix.log.QueryLogger;
 import org.apache.phoenix.monitoring.MetricType;
 import org.apache.phoenix.monitoring.OverAllQueryMetrics;
 import org.apache.phoenix.monitoring.ReadMetricQueue;
@@ -72,6 +75,9 @@ import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.SQLCloseable;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
 
 
 
@@ -122,6 +128,14 @@ public class PhoenixResultSet implements ResultSet, SQLCloseable {
     private boolean isClosed = false;
     private boolean wasNull = false;
     private boolean firstRecordRead = false;
+
+    private QueryLogger queryLogger;
+
+    private Long count = 0L;
+
+    private QueryLogState logStatus = QueryLogState.COMPLETED;
+
+    private RuntimeException exception;
     
     public PhoenixResultSet(ResultIterator resultIterator, RowProjector rowProjector, StatementContext ctx) throws SQLException {
         this.rowProjector = rowProjector;
@@ -130,6 +144,7 @@ public class PhoenixResultSet implements ResultSet, SQLCloseable {
         this.statement = context.getStatement();
         this.readMetricsQueue = context.getReadMetricsQueue();
         this.overAllQueryMetrics = context.getOverallQueryMetrics();
+        this.queryLogger = context.getQueryLogger();
     }
     
     @Override
@@ -779,17 +794,39 @@ public class PhoenixResultSet implements ResultSet, SQLCloseable {
             currentRow = scanner.next();
             if (currentRow == null) {
                 close();
+            }else{
+                count++;
             }
             rowProjector.reset();
         } catch (RuntimeException e) {
+            this.logStatus=QueryLogState.FAILED;
             // FIXME: Expression.evaluate does not throw SQLException
             // so this will unwrap throws from that.
+            this.exception = e;
             if (e.getCause() instanceof SQLException) {
                 throw (SQLException) e.getCause();
             }
             throw e;
+        }finally{
+            if (currentRow == null && queryLogger != null ) {
+                if (queryLogger.isDebugEnabled()) {
+                    Builder<QueryLogInfo, Object> queryLogBuilder = ImmutableMap.builder();
+                    queryLogBuilder.put(QueryLogInfo.NO_OF_RESULTS_ITERATED_I, count);
+                    queryLogBuilder.put(QueryLogInfo.TOTAL_EXECUTION_TIME_I,
+                            System.currentTimeMillis() - queryLogger.getStartTime());
+                    queryLogBuilder.put(QueryLogInfo.SCAN_METRICS_JSON_I,
+                            readMetricsQueue.getScanMetricsHolderList().toString());
+                    if (this.exception != null) {
+                        queryLogBuilder.put(QueryLogInfo.EXCEPTION_TRACE_I,
+                                Throwables.getStackTraceAsString(this.exception));
+                    }
+                    readMetricsQueue.getScanMetricsHolderList().clear();
+                    queryLogger.log(logStatus, queryLogBuilder.build());
+                }
+            }
         }
         if (currentRow == null) {
+            
             overAllQueryMetrics.endQuery();
             overAllQueryMetrics.stopResultSetWatch();
         }
@@ -1301,6 +1338,7 @@ public class PhoenixResultSet implements ResultSet, SQLCloseable {
     
     public void resetMetrics() {
         readMetricsQueue.clearMetrics();
+        readMetricsQueue.getScanMetricsHolderList().clear();
         overAllQueryMetrics.reset();
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 4a692d3..f526419 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
@@ -92,6 +92,10 @@ import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.iterate.MaterializedResultIterator;
 import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.log.QueryLogInfo;
+import org.apache.phoenix.log.QueryLogState;
+import org.apache.phoenix.log.QueryLogger;
+import org.apache.phoenix.log.QueryLoggerUtil;
 import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.AddJarsStatement;
@@ -186,6 +190,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.math.IntMath;
@@ -269,26 +275,19 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         return new PhoenixResultSet(iterator, projector, context);
     }
     
-    protected boolean execute(final CompilableStatement stmt) throws SQLException {
-        if (stmt.getOperation().isMutation()) {
-            executeMutation(stmt);
-            return false;
-        }
-        executeQuery(stmt);
-        return true;
-    }
-    
     protected QueryPlan optimizeQuery(CompilableStatement stmt) throws SQLException {
         QueryPlan plan = stmt.compilePlan(this, Sequence.ValueOp.VALIDATE_SEQUENCE);
         return connection.getQueryServices().getOptimizer().optimize(this, plan);
     }
     
-    protected PhoenixResultSet executeQuery(final CompilableStatement stmt) throws SQLException {
-      return executeQuery(stmt,true);
+    protected PhoenixResultSet executeQuery(final CompilableStatement stmt, final QueryLogger queryLogger)
+            throws SQLException {
+        return executeQuery(stmt, true, queryLogger);
     }
     private PhoenixResultSet executeQuery(final CompilableStatement stmt,
-        final boolean doRetryOnMetaNotFoundError) throws SQLException {
+        final boolean doRetryOnMetaNotFoundError, final QueryLogger queryLogger) throws SQLException {
         GLOBAL_SELECT_SQL_COUNTER.increment();
+        
         try {
             return CallRunner.run(
                 new CallRunner.CallableThrowable<PhoenixResultSet, SQLException>() {
@@ -297,6 +296,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                     final long startTime = System.currentTimeMillis();
                     try {
                         PhoenixConnection conn = getConnection();
+                        
                         if (conn.getQueryServices().isUpgradeRequired() && !conn.isRunningUpgrade()
                                 && stmt.getOperation() != Operation.UPGRADE) {
                             throw new UpgradeRequiredException();
@@ -317,6 +317,13 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                             logger.debug(LogUtil.addCustomAnnotations("Explain plan: " + explainPlan, connection));
                         }
                         StatementContext context = plan.getContext();
+                        context.setQueryLogger(queryLogger);
+                        if(queryLogger.isDebugEnabled()){
+                            Builder<QueryLogInfo, Object> queryLogBuilder = ImmutableMap.builder();
+                            queryLogBuilder.put(QueryLogInfo.EXPLAIN_PLAN_I, QueryUtil.getExplainPlan(resultIterator));
+                            queryLogBuilder.put(QueryLogInfo.GLOBAL_SCAN_DETAILS_I, context.getScan()!=null?context.getScan().toString():null);
+                            queryLogger.log(QueryLogState.COMPILED, queryLogBuilder.build());
+                        }
                         context.getOverallQueryMetrics().startQuery();
                         PhoenixResultSet rs = newResultSet(resultIterator, plan.getProjector(), plan.getContext());
                         resultSets.add(rs);
@@ -338,7 +345,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                                 logger.debug("Reloading table "+ e.getTableName()+" data from server");
                             if(new MetaDataClient(connection).updateCache(connection.getTenantId(),
                                 e.getSchemaName(), e.getTableName(), true).wasUpdated()){
-                                return executeQuery(stmt, false);
+                                //TODO we can log retry count and error for debugging in LOG table
+                                return executeQuery(stmt, false, queryLogger);
                             }
                         }
                         throw e;
@@ -358,6 +366,13 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                 }
                 }, PhoenixContextExecutor.inContext());
         }catch (Exception e) {
+            if (queryLogger.isDebugEnabled()) {
+                Builder<QueryLogInfo, Object> queryLogBuilder = ImmutableMap.builder();
+                queryLogBuilder.put(QueryLogInfo.TOTAL_EXECUTION_TIME_I,
+                        System.currentTimeMillis() - queryLogger.getStartTime());
+                queryLogBuilder.put(QueryLogInfo.EXCEPTION_TRACE_I, Throwables.getStackTraceAsString(e));
+                queryLogger.log(QueryLogState.FAILED, queryLogBuilder.build());
+            }
             Throwables.propagateIfInstanceOf(e, SQLException.class);
             Throwables.propagate(e);
             throw new IllegalStateException(); // Can't happen as Throwables.propagate() always throws
@@ -1750,16 +1765,37 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         return compileMutation(stmt, sql);
     }
 
+    public QueryLogger createQueryLogger(CompilableStatement stmt, String sql) throws SQLException {
+        boolean isSystemTable=false;
+        if(stmt instanceof ExecutableSelectStatement){
+            TableNode from = ((ExecutableSelectStatement)stmt).getFrom();
+            if(from instanceof NamedTableNode){
+                String schemaName = ((NamedTableNode)from).getName().getSchemaName();
+                if(schemaName==null){
+                    schemaName=connection.getSchema();
+                }
+                if(PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA.equals(schemaName)){
+                    isSystemTable=true;
+                }
+            }
+        }
+        QueryLogger queryLogger = QueryLogger.getInstance(connection,isSystemTable);
+        QueryLoggerUtil.logInitialDetails(queryLogger, connection.getTenantId(),
+                connection.getQueryServices(), sql, queryLogger.getStartTime(), getParameters());
+        return queryLogger;
+    }
+    
     @Override
     public ResultSet executeQuery(String sql) throws SQLException {
         if (logger.isDebugEnabled()) {
             logger.debug(LogUtil.addCustomAnnotations("Execute query: " + sql, connection));
         }
+        
         CompilableStatement stmt = parseStatement(sql);
         if (stmt.getOperation().isMutation()) {
             throw new ExecuteQueryNotApplicableException(sql);
         }
-        return executeQuery(stmt);
+        return executeQuery(stmt,createQueryLogger(stmt,sql));
     }
 
     @Override
@@ -1795,7 +1831,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
             flushIfNecessary();
             return false;
         }
-        executeQuery(stmt);
+        
+        executeQuery(stmt,createQueryLogger(stmt,sql));
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/LogLevel.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/LogLevel.java b/phoenix-core/src/main/java/org/apache/phoenix/log/LogLevel.java
new file mode 100644
index 0000000..5792658
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/LogLevel.java
@@ -0,0 +1,22 @@
+/*
+ * 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.log;
+
+public enum LogLevel {
+    OFF, INFO, DEBUG, TRACE
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.java
new file mode 100644
index 0000000..817f9ec
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.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.log;
+
+import java.io.IOException;
+import java.sql.SQLException;
+
+/**
+ * Used by the event handler to write RingBufferEvent, this is done in a seperate thread from the application configured
+ * during disruptor
+ */
+public interface LogWriter {
+    /**
+     * Called by ring buffer event handler to write RingBufferEvent
+     * 
+     * @param event
+     * @throws SQLException
+     * @throws IOException
+     */
+    void write(RingBufferEvent event) throws SQLException, IOException;
+
+    /**
+     * will be called when disruptor is getting shutdown
+     * 
+     * @throws IOException
+     */
+
+    void close() throws IOException;
+
+    /**
+     * if writer is closed and cannot write further event
+     * 
+     * @return
+     */
+    boolean isClosed();
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogDetailsEventHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogDetailsEventHandler.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogDetailsEventHandler.java
new file mode 100644
index 0000000..ee6b2d6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogDetailsEventHandler.java
@@ -0,0 +1,63 @@
+/*
+ * 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.log;
+
+import java.sql.SQLException;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.lmax.disruptor.LifecycleAware;
+import com.lmax.disruptor.Sequence;
+import com.lmax.disruptor.SequenceReportingEventHandler;
+
+
+public class QueryLogDetailsEventHandler implements SequenceReportingEventHandler<RingBufferEvent>, LifecycleAware {
+    private Sequence sequenceCallback;
+    private LogWriter logWriter;
+
+    public QueryLogDetailsEventHandler(Configuration configuration) throws SQLException{
+        this.logWriter = new TableLogWriter(configuration);
+    }
+    
+    @Override
+    public void setSequenceCallback(final Sequence sequenceCallback) {
+        this.sequenceCallback = sequenceCallback;
+    }
+
+    @Override
+    public void onEvent(final RingBufferEvent event, final long sequence, final boolean endOfBatch) throws Exception {
+        logWriter.write(event);
+        event.clear();
+    }
+
+    @Override
+    public void onStart() {
+    }
+
+    @Override
+    public void onShutdown() {
+        try {
+            if (logWriter != null) {
+                logWriter.close();
+            }
+        } catch (Exception e) {
+            //Ignore
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogInfo.java
new file mode 100644
index 0000000..87de267
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogInfo.java
@@ -0,0 +1,87 @@
+/**
+ * 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.log;
+
+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.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TOTAL_EXECUTION_TIME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USER;
+
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PVarchar;
+
+
+public enum QueryLogInfo {
+    
+    CLIENT_IP_I(CLIENT_IP, QueryLogState.STARTED, LogLevel.INFO, PVarchar.INSTANCE),
+    QUERY_I(QUERY,QueryLogState.STARTED, LogLevel.INFO,PVarchar.INSTANCE),
+    BIND_PARAMETERS_I(BIND_PARAMETERS,QueryLogState.STARTED, LogLevel.TRACE,PVarchar.INSTANCE),
+    QUERY_ID_I(QUERY_ID,QueryLogState.STARTED, LogLevel.INFO,PVarchar.INSTANCE),
+    TENANT_ID_I(TENANT_ID,QueryLogState.STARTED, LogLevel.INFO,PVarchar.INSTANCE),
+    START_TIME_I(START_TIME,QueryLogState.STARTED, LogLevel.INFO,PTimestamp.INSTANCE),
+    USER_I(USER,QueryLogState.STARTED, LogLevel.INFO,PVarchar.INSTANCE),
+    EXPLAIN_PLAN_I(EXPLAIN_PLAN,QueryLogState.COMPILED, LogLevel.DEBUG,PVarchar.INSTANCE),
+    GLOBAL_SCAN_DETAILS_I(GLOBAL_SCAN_DETAILS,QueryLogState.COMPILED, LogLevel.DEBUG,PVarchar.INSTANCE),
+    NO_OF_RESULTS_ITERATED_I(NO_OF_RESULTS_ITERATED,QueryLogState.COMPLETED, LogLevel.DEBUG,PLong.INSTANCE),
+    EXCEPTION_TRACE_I(EXCEPTION_TRACE,QueryLogState.COMPLETED, LogLevel.DEBUG,PVarchar.INSTANCE),
+    QUERY_STATUS_I(QUERY_STATUS,QueryLogState.COMPLETED, LogLevel.DEBUG,PVarchar.INSTANCE),
+    TOTAL_EXECUTION_TIME_I(TOTAL_EXECUTION_TIME,QueryLogState.COMPLETED, LogLevel.DEBUG,PLong.INSTANCE),
+    SCAN_METRICS_JSON_I(SCAN_METRICS_JSON,QueryLogState.COMPLETED, LogLevel.DEBUG,PVarchar.INSTANCE);
+    
+    public final String columnName;
+    public final QueryLogState logState;
+    public final LogLevel logLevel;
+    public final PDataType dataType;
+
+    private QueryLogInfo(String columnName, QueryLogState logState, LogLevel logLevel, PDataType dataType) {
+        this.columnName = columnName;
+        this.logState=logState;
+        this.logLevel=logLevel;
+        this.dataType=dataType;
+    }
+
+    public String getColumnName() {
+        return columnName;
+    }
+
+    public QueryLogState getLogState() {
+        return logState;
+    }
+
+    public LogLevel getLogLevel() {
+        return logLevel;
+    }
+
+    public PDataType getDataType() {
+        return dataType;
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogState.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogState.java
new file mode 100644
index 0000000..e27f0e8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogState.java
@@ -0,0 +1,22 @@
+/*
+ * 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.log;
+
+public enum QueryLogState {
+    STARTED, PLAN, COMPILED, EXECUTION, COMPLETED,FAILED 
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogger.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogger.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogger.java
new file mode 100644
index 0000000..b2fb235
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLogger.java
@@ -0,0 +1,145 @@
+/*
+ * 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.log;
+
+import java.util.UUID;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+
+import com.google.common.collect.ImmutableMap;
+
+import io.netty.util.internal.ThreadLocalRandom;
+
+/*
+ * Wrapper for query translator
+ */
+public class QueryLogger {
+    private final ThreadLocal<RingBufferEventTranslator> threadLocalTranslator = new ThreadLocal<>();
+    private QueryLoggerDisruptor queryDisruptor;
+    private String queryId;
+    private Long startTime;
+    private LogLevel logLevel;
+    private static final Log LOG = LogFactory.getLog(QueryLoggerDisruptor.class);
+    
+    private QueryLogger(PhoenixConnection connection) {
+        this.queryId = UUID.randomUUID().toString();
+        this.queryDisruptor = connection.getQueryServices().getQueryDisruptor();
+        this.startTime = System.currentTimeMillis();
+        logLevel = connection.getLogLevel();
+    }
+    
+    private QueryLogger() {
+        logLevel = LogLevel.OFF;
+    }
+    
+    private RingBufferEventTranslator getCachedTranslator() {
+        RingBufferEventTranslator result = threadLocalTranslator.get();
+        if (result == null) {
+            result = new RingBufferEventTranslator(queryId);
+            threadLocalTranslator.set(result);
+        }
+        return result;
+    }
+    
+    private static final QueryLogger NO_OP_INSTANCE = new QueryLogger() {
+        @Override
+        public void log(QueryLogState logState, ImmutableMap<QueryLogInfo, Object> map) {
+
+        }
+        
+        @Override
+        public boolean isDebugEnabled(){
+            return false;
+        }
+        
+        @Override
+        public boolean isInfoEnabled(){
+            return false;
+        }
+    };
+
+    public static QueryLogger getInstance(PhoenixConnection connection, boolean isSystemTable) {
+        if (connection.getLogLevel() == LogLevel.OFF || isSystemTable || ThreadLocalRandom.current()
+                .nextDouble() > connection.getLogSamplingRate()) { return NO_OP_INSTANCE; }
+        return new QueryLogger(connection);
+    }
+
+    /**
+     * Add query log in the table, columns will be logged depending upon the connection logLevel
+     * @param logState State of the query
+     * @param map Value of the map should be in format of the corresponding data type 
+     */
+    public void log(QueryLogState logState, ImmutableMap<QueryLogInfo, Object> map) {
+        final RingBufferEventTranslator translator = getCachedTranslator();
+        translator.setQueryInfo(logState, map, logLevel);
+        publishLogs(translator);
+    }
+    
+    private boolean publishLogs(RingBufferEventTranslator translator) {
+        if (queryDisruptor == null) { return false; }
+        boolean isLogged = queryDisruptor.tryPublish(translator);
+        if (!isLogged && LOG.isDebugEnabled()) {
+            LOG.debug("Unable to write query log in table as ring buffer queue is full!!");
+        }
+        return isLogged;
+    }
+
+    /**
+     * Start time when the logger was started, if {@link LogLevel#OFF} then it's the current time
+     */
+    public Long getStartTime() {
+        return startTime != null ? startTime : System.currentTimeMillis();
+    }
+    
+    /**
+     *  Is debug logging currently enabled?
+     *  Call this method to prevent having to perform expensive operations (for example, String concatenation) when the log level is more than debug.
+     */
+    public boolean isDebugEnabled(){
+        return isLevelEnabled(LogLevel.DEBUG);
+    }
+    
+    private boolean isLevelEnabled(LogLevel logLevel){
+        return this.logLevel != null ? logLevel.ordinal() <= this.logLevel.ordinal() : false;
+    }
+    
+    /**
+     * Is Info logging currently enabled?
+     * Call this method to prevent having to perform expensive operations (for example, String concatenation) when the log level is more than info.
+     * @return
+     */
+    public boolean isInfoEnabled(){
+        return isLevelEnabled(LogLevel.INFO);
+    }
+
+    /**
+     * Return queryId of the current query logger , needed by the application 
+     * to correlate with the logging table.
+     * Eg(usage):-
+     * StatementContext context = ((PhoenixResultSet)rs).getContext();
+     * String queryId = context.getQueryLogger().getQueryId();
+     * 
+     * @return
+     */
+    public String getQueryId() {
+        return this.queryId;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDefaultExceptionHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDefaultExceptionHandler.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDefaultExceptionHandler.java
new file mode 100644
index 0000000..e9ae6bd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDefaultExceptionHandler.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.log;
+
+import com.lmax.disruptor.ExceptionHandler;
+
+class QueryLoggerDefaultExceptionHandler implements ExceptionHandler<RingBufferEvent> {
+
+    @Override
+    public void handleEventException(Throwable ex, long sequence, RingBufferEvent event) {
+        final StringBuilder sb = new StringBuilder(512);
+        sb.append("Query Logger error handling event seq=").append(sequence).append(", value='");
+        try {
+            sb.append(event);
+        } catch (final Exception ignored) {
+            sb.append("[ERROR calling ").append(event.getClass()).append(".toString(): ");
+            sb.append(ignored).append("]");
+        }
+        sb.append("':");
+        System.err.println(sb);
+        ex.printStackTrace();
+    }
+
+    @Override
+    public void handleOnStartException(final Throwable throwable) {
+        System.err.println("QueryLogger error starting:");
+        throwable.printStackTrace();
+    }
+
+    @Override
+    public void handleOnShutdownException(final Throwable throwable) {
+        System.err.println("QueryLogger error shutting down:");
+        throwable.printStackTrace();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDisruptor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDisruptor.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDisruptor.java
new file mode 100644
index 0000000..b548d6c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerDisruptor.java
@@ -0,0 +1,117 @@
+/*
+ * 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.log;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.query.QueryServices;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventTranslator;
+import com.lmax.disruptor.ExceptionHandler;
+import com.lmax.disruptor.TimeoutException;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+public class QueryLoggerDisruptor implements Closeable{
+    
+    private volatile Disruptor<RingBufferEvent> disruptor;
+    private boolean isClosed = false;
+    //number of elements to create within the ring buffer.
+    private static final int RING_BUFFER_SIZE = 256 * 1024;
+    private static final Log LOG = LogFactory.getLog(QueryLoggerDisruptor.class);
+    private static final String DEFAULT_WAIT_STRATEGY = BlockingWaitStrategy.class.getName();
+    
+    public QueryLoggerDisruptor(Configuration configuration) throws SQLException{
+        WaitStrategy waitStrategy;
+        try {
+            waitStrategy = (WaitStrategy)Class
+                    .forName(configuration.get(QueryServices.LOG_BUFFER_WAIT_STRATEGY, DEFAULT_WAIT_STRATEGY)).newInstance();
+        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+            throw new SQLException(e); 
+        }
+        
+        ThreadFactory threadFactory = new ThreadFactoryBuilder()
+                .setNameFormat("QueryLogger" + "-thread-%s")
+                .setDaemon(true)
+                .setThreadFactory(new ThreadFactory() {
+                    @Override
+                    public Thread newThread(Runnable r) {
+                        final Thread result = Executors.defaultThreadFactory().newThread(r);
+                        result.setContextClassLoader(QueryLoggerDisruptor.class.getClass().getClassLoader());
+                        return result;
+                    }
+                })
+                .build();
+        disruptor = new Disruptor<RingBufferEvent>(RingBufferEvent.FACTORY,
+                configuration.getInt(QueryServices.LOG_BUFFER_SIZE, RING_BUFFER_SIZE), threadFactory, ProducerType.MULTI,
+                waitStrategy);
+        final ExceptionHandler<RingBufferEvent> errorHandler = new QueryLoggerDefaultExceptionHandler();
+        disruptor.setDefaultExceptionHandler(errorHandler);
+
+        final QueryLogDetailsEventHandler[] handlers = { new QueryLogDetailsEventHandler(configuration) };
+        disruptor.handleEventsWith(handlers);
+        LOG.info("Starting  QueryLoggerDisruptor for with ringbufferSize=" + disruptor.getRingBuffer().getBufferSize()
+                + ", waitStrategy=" + waitStrategy.getClass().getSimpleName() + ", " + "exceptionHandler="
+                + errorHandler + "...");
+        disruptor.start();
+        
+    }
+    
+    /**
+     * Attempts to publish an event by translating (write) data representations into events claimed from the RingBuffer.
+     * @param translator
+     * @return
+     */
+    public boolean tryPublish(final EventTranslator<RingBufferEvent> translator) {
+        if(isClosed()){
+            return false;
+        }
+        return disruptor.getRingBuffer().tryPublishEvent(translator);
+    }
+    
+
+    public boolean isClosed() {
+        return isClosed ;
+    }
+
+    @Override
+    public void close() throws IOException {
+        isClosed = true;
+        LOG.info("Shutting down QueryLoggerDisruptor..");
+        try {
+            //we can wait for 2 seconds, so that backlog can be committed
+            disruptor.shutdown(2, TimeUnit.SECONDS);
+        } catch (TimeoutException e) {
+            throw new IOException(e);
+        }
+
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerUtil.java
new file mode 100644
index 0000000..2f22931
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/QueryLoggerUtil.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.log;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.schema.PName;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
+
+public class QueryLoggerUtil {
+
+    public static void logInitialDetails(QueryLogger queryLogger, PName tenantId,
+            ConnectionQueryServices queryServices, String query, long startTime, List<Object> bindParameters) {
+        queryLogger.log(QueryLogState.STARTED,
+                getInitialDetails(tenantId, queryServices, query, startTime, bindParameters));
+
+    }
+
+    private static ImmutableMap<QueryLogInfo, Object> getInitialDetails(PName tenantId,
+            ConnectionQueryServices queryServices, String query, long startTime, List<Object> bindParameters) {
+        Builder<QueryLogInfo, Object> queryLogBuilder = ImmutableMap.builder();
+        String clientIP;
+        try {
+            clientIP = InetAddress.getLocalHost().getHostAddress();
+        } catch (UnknownHostException e) {
+            clientIP = "UnknownHost";
+        }
+        queryLogBuilder.put(QueryLogInfo.CLIENT_IP_I, clientIP);
+        queryLogBuilder.put(QueryLogInfo.QUERY_I, query);
+        queryLogBuilder.put(QueryLogInfo.START_TIME_I, startTime);
+        if (bindParameters != null) {
+            queryLogBuilder.put(QueryLogInfo.BIND_PARAMETERS_I, StringUtils.join(bindParameters,","));
+        }
+        if (tenantId != null) {
+            queryLogBuilder.put(QueryLogInfo.TENANT_ID_I, tenantId.getString());
+        }
+        queryLogBuilder.put(QueryLogInfo.USER_I, queryServices.getUserName() != null ? queryServices.getUserName()
+                : queryServices.getUser().getShortName());
+        return queryLogBuilder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEvent.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEvent.java b/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEvent.java
new file mode 100644
index 0000000..96e4bf9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEvent.java
@@ -0,0 +1,93 @@
+/*
+ * 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.log;
+
+import com.google.common.collect.ImmutableMap;
+import com.lmax.disruptor.EventFactory;
+
+ class RingBufferEvent {
+    private String queryId;
+    private QueryLogState logState;
+    private LogLevel connectionLogLevel;
+    private ImmutableMap<QueryLogInfo, Object> queryInfo;
+    
+    public static final Factory FACTORY = new Factory();
+    
+    /**
+     * Creates the events that will be put in the RingBuffer.
+     */
+    private static class Factory implements EventFactory<RingBufferEvent> {
+        @Override
+        public RingBufferEvent newInstance() {
+            final RingBufferEvent result = new RingBufferEvent();
+            return result;
+        }
+    }
+
+    public void clear() {
+        this.logState=null;
+        this.queryInfo=null;
+        this.queryId=null;
+    }
+
+   
+    public String getQueryId() {
+        return queryId;
+    }
+
+    public static Factory getFactory() {
+        return FACTORY;
+    }
+    
+    public QueryLogState getLogState() {
+        return logState;
+    }
+
+    public void setQueryInfo(ImmutableMap<QueryLogInfo, Object> queryInfo) {
+        this.queryInfo=queryInfo;
+        
+    }
+
+    public void setQueryId(String queryId) {
+        this.queryId=queryId;
+        
+    }
+
+    public ImmutableMap<QueryLogInfo, Object> getQueryInfo() {
+        return queryInfo;
+        
+    }
+
+    public void setLogState(QueryLogState logState) {
+        this.logState=logState;
+        
+    }
+
+
+    public LogLevel getConnectionLogLevel() {
+        return connectionLogLevel;
+    }
+
+
+    public void setConnectionLogLevel(LogLevel connectionLogLevel) {
+        this.connectionLogLevel = connectionLogLevel;
+    }
+
+    
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEventTranslator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEventTranslator.java b/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEventTranslator.java
new file mode 100644
index 0000000..653ddd6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/RingBufferEventTranslator.java
@@ -0,0 +1,53 @@
+/*
+ * 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.log;
+
+import com.google.common.collect.ImmutableMap;
+import com.lmax.disruptor.EventTranslator;
+
+class RingBufferEventTranslator implements EventTranslator<RingBufferEvent> {
+    private String queryId;
+    private QueryLogState logState;
+    private ImmutableMap<QueryLogInfo, Object> queryInfo;
+    private LogLevel connectionLogLevel;
+    
+    public RingBufferEventTranslator(String queryId) {
+        this.queryId=queryId;
+    }
+
+    @Override
+    public void translateTo(RingBufferEvent event, long sequence) {
+        event.setQueryId(queryId);
+        event.setQueryInfo(queryInfo);
+        event.setLogState(logState);
+        event.setConnectionLogLevel(connectionLogLevel);
+        clear();
+    }
+
+    private void clear() {
+        setQueryInfo(null,null,null);
+    }
+   
+    public void setQueryInfo(QueryLogState logState, ImmutableMap<QueryLogInfo, Object> queryInfo,
+            LogLevel connectionLogLevel) {
+        this.queryInfo = queryInfo;
+        this.logState = logState;
+        this.connectionLogLevel = connectionLogLevel;
+    }
+
+}


[10/21] phoenix git commit: PHOENIX-4366 Rebuilding a local index fails sometimes

Posted by pb...@apache.org.
PHOENIX-4366 Rebuilding a local index fails sometimes


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

Branch: refs/heads/4.x-cdh5.12
Commit: a874df3a70bb52ea56f0159c37cdbb16534d762f
Parents: 0987d09
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Apr 11 21:37:45 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:27:33 2018 +0100

----------------------------------------------------------------------
 .../coprocessor/BaseScannerRegionObserver.java  |  7 +----
 .../GroupedAggregateRegionObserver.java         |  4 +++
 .../phoenix/coprocessor/ScanRegionObserver.java |  4 ++-
 .../UngroupedAggregateRegionObserver.java       |  7 ++++-
 .../NonAggregateRegionScannerFactory.java       | 29 +++++++++-----------
 .../phoenix/iterate/RegionScannerFactory.java   |  4 +--
 .../apache/phoenix/iterate/SnapshotScanner.java |  6 +---
 7 files changed, 30 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/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 7ef64b0..3310131 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
@@ -139,8 +139,6 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
 
     /** Exposed for testing */
     public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server";
-    protected QualifierEncodingScheme encodingScheme;
-    protected boolean useNewValueColumnQualifier;
 
     @Override
     public void start(CoprocessorEnvironment e) throws IOException {
@@ -211,8 +209,6 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             // start exclusive and the stop inclusive.
             ScanUtil.setupReverseScan(scan);
         }
-        this.encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
-        this.useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
         return s;
     }
 
@@ -350,8 +346,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             final byte[][] viewConstants, final TupleProjector projector,
             final ImmutableBytesWritable ptr, final boolean useQualiferAsListIndex) {
 
-        RegionScannerFactory regionScannerFactory = new NonAggregateRegionScannerFactory(c.getEnvironment(),
-            useNewValueColumnQualifier, encodingScheme);
+        RegionScannerFactory regionScannerFactory = new NonAggregateRegionScannerFactory(c.getEnvironment());
 
         return regionScannerFactory.getWrappedScanner(c.getEnvironment(), s, null, null, offset, scan, dataColumns, tupleProjector,
                 dataRegion, indexMaintainer, null, viewConstants, null, null, projector, ptr, useQualiferAsListIndex);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 67cc114..201bcec 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -62,6 +62,7 @@ import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.EncodedColumnQualiferCellsList;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
@@ -110,6 +111,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             keyOrdered = true;
         }
         int offset = 0;
+        boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
         if (ScanUtil.isLocalIndex(scan)) {
             /*
              * For local indexes, we need to set an offset on row key expressions to skip
@@ -395,6 +397,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         boolean useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan));
         final boolean spillableEnabled =
                 conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
+        final PTable.QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
 
         GroupByCache groupByCache =
                 GroupByCacheFactory.INSTANCE.newCache(
@@ -466,6 +469,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         }
         final Pair<Integer, Integer> minMaxQualifiers = EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan);
         final boolean useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(minMaxQualifiers);
+        final PTable.QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
         return new BaseRegionScanner(scanner) {
             private long rowCount = 0;
             private ImmutableBytesPtr currentKey = null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index b006ef6..2d9cd4f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.OrderByExpression;
 import org.apache.phoenix.iterate.NonAggregateRegionScannerFactory;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 
 /**
  *
@@ -68,7 +70,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
 
     @Override
     protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws Throwable {
-        NonAggregateRegionScannerFactory nonAggregateROUtil = new NonAggregateRegionScannerFactory(c.getEnvironment(), useNewValueColumnQualifier, encodingScheme);
+        NonAggregateRegionScannerFactory nonAggregateROUtil = new NonAggregateRegionScannerFactory(c.getEnvironment());
         return nonAggregateROUtil.getRegionScanner(scan, s);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/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 27d3880..de57772 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
@@ -376,7 +376,12 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                     env, region.getRegionInfo().getTable().getNameAsString(), ts,
                     gp_width_bytes, gp_per_region_bytes);
             return collectStats(s, statsCollector, region, scan, env.getConfiguration());
-        } else if (ScanUtil.isIndexRebuild(scan)) { return rebuildIndices(s, region, scan, env.getConfiguration()); }
+        } else if (ScanUtil.isIndexRebuild(scan)) {
+            return rebuildIndices(s, region, scan, env.getConfiguration());
+        }
+
+        PTable.QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
+        boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
         int offsetToBe = 0;
         if (localIndexScan) {
             /*

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/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 91bab6f..90ea025 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
@@ -73,20 +73,13 @@ import static org.apache.phoenix.util.EncodedColumnsUtil.getMinMaxQualifiersFrom
 
 public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
 
-  private ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-  private KeyValueSchema kvSchema = null;
-  private ValueBitSet kvSchemaBitSet;
-
-  public NonAggregateRegionScannerFactory(RegionCoprocessorEnvironment env, boolean useNewValueColumnQualifier,
-      PTable.QualifierEncodingScheme encodingScheme) {
+  public NonAggregateRegionScannerFactory(RegionCoprocessorEnvironment env) {
     this.env = env;
-    this.useNewValueColumnQualifier = useNewValueColumnQualifier;
-    this.encodingScheme = encodingScheme;
   }
 
   @Override
   public RegionScanner getRegionScanner(final Scan scan, final RegionScanner s) throws Throwable {
-
+      ImmutableBytesWritable ptr = new ImmutableBytesWritable();
     int offset = 0;
     if (ScanUtil.isLocalIndex(scan)) {
             /*
@@ -105,9 +98,17 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
       scanOffset = (Integer)PInteger.INSTANCE.toObject(scanOffsetBytes);
     }
     RegionScanner innerScanner = s;
+    PTable.QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
+    boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
 
     Set<KeyValueColumnExpression> arrayKVRefs = Sets.newHashSet();
-    Expression[] arrayFuncRefs = deserializeArrayPostionalExpressionInfoFromScan(scan, innerScanner, arrayKVRefs);
+    Expression[] arrayFuncRefs = deserializeArrayPositionalExpressionInfoFromScan(scan, innerScanner, arrayKVRefs);
+    KeyValueSchema.KeyValueSchemaBuilder builder = new KeyValueSchema.KeyValueSchemaBuilder(0);
+    for (Expression expression : arrayFuncRefs) {
+        builder.addField(expression);
+    }
+    KeyValueSchema kvSchema = builder.build();
+    ValueBitSet kvSchemaBitSet = ValueBitSet.newInstance(kvSchema);
     TupleProjector tupleProjector = null;
     Region dataRegion = null;
     IndexMaintainer indexMaintainer = null;
@@ -195,13 +196,12 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
     }
   }
 
-  private Expression[] deserializeArrayPostionalExpressionInfoFromScan(Scan scan, RegionScanner s,
-      Set<KeyValueColumnExpression> arrayKVRefs) {
+  private Expression[] deserializeArrayPositionalExpressionInfoFromScan(Scan scan, RegionScanner s,
+                                                                        Set<KeyValueColumnExpression> arrayKVRefs) {
     byte[] specificArrayIdx = scan.getAttribute(BaseScannerRegionObserver.SPECIFIC_ARRAY_INDEX);
     if (specificArrayIdx == null) {
       return null;
     }
-    KeyValueSchema.KeyValueSchemaBuilder builder = new KeyValueSchema.KeyValueSchemaBuilder(0);
     ByteArrayInputStream stream = new ByteArrayInputStream(specificArrayIdx);
     try {
       DataInputStream input = new DataInputStream(stream);
@@ -219,10 +219,7 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
         ArrayIndexFunction arrayIdxFunc = new ArrayIndexFunction();
         arrayIdxFunc.readFields(input);
         arrayFuncRefs[i] = arrayIdxFunc;
-        builder.addField(arrayIdxFunc);
       }
-      kvSchema = builder.build();
-      kvSchemaBitSet = ValueBitSet.newInstance(kvSchema);
       return arrayFuncRefs;
     } catch (IOException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerFactory.java
index 3dcbef9..aed5805 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerFactory.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.ValueBitSet;
 import org.apache.phoenix.schema.tuple.*;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
@@ -54,8 +55,6 @@ import java.util.Set;
 public abstract class RegionScannerFactory {
 
   protected RegionCoprocessorEnvironment env;
-  protected boolean useNewValueColumnQualifier;
-  protected PTable.QualifierEncodingScheme encodingScheme;
 
   /**
    * Returns the region based on the value of the
@@ -107,6 +106,7 @@ public abstract class RegionScannerFactory {
       private boolean hasReferences = checkForReferenceFiles();
       private HRegionInfo regionInfo = env.getRegionInfo();
       private byte[] actualStartKey = getActualStartKey();
+      private boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
 
       // If there are any reference files after local index region merge some cases we might
       // get the records less than scan start row key. This will happen when we replace the

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a874df3a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
index 68592ef..9e2a08b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
@@ -54,15 +54,11 @@ public class SnapshotScanner extends AbstractClientScanner {
     values = new ArrayList<>();
     this.region = HRegion.openHRegion(conf, fs, rootDir, hri, htd, null, null, null);
 
-    // process the region scanner for non-aggregate queries
-    PTable.QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
-    boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
-
     RegionCoprocessorEnvironment snapshotEnv = getSnapshotContextEnvironment(conf);
 
     RegionScannerFactory regionScannerFactory;
     if (scan.getAttribute(BaseScannerRegionObserver.NON_AGGREGATE_QUERY) != null) {
-      regionScannerFactory = new NonAggregateRegionScannerFactory(snapshotEnv, useNewValueColumnQualifier, encodingScheme);
+      regionScannerFactory = new NonAggregateRegionScannerFactory(snapshotEnv);
     } else {
       /* future work : Snapshot M/R jobs for aggregate queries*/
       throw new UnsupportedOperationException("Snapshot M/R jobs not available for aggregate queries");


[21/21] phoenix git commit: Changes for CDH 5.12.x

Posted by pb...@apache.org.
Changes for CDH 5.12.x


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

Branch: refs/heads/4.x-cdh5.12
Commit: 157139688eb422d7c0196d762afa2f937e18e237
Parents: 9fc11de
Author: Pedro Boado <pb...@apache.org>
Authored: Sat Mar 10 17:54:04 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Sat Apr 14 08:27:41 2018 +0100

----------------------------------------------------------------------
 phoenix-assembly/pom.xml                        |  2 +-
 phoenix-client/pom.xml                          |  2 +-
 phoenix-core/pom.xml                            |  2 +-
 .../hadoop/hbase/ipc/PhoenixRpcScheduler.java   | 34 ++++++++++++++++++--
 phoenix-flume/pom.xml                           |  2 +-
 phoenix-hive/pom.xml                            |  2 +-
 phoenix-kafka/pom.xml                           |  2 +-
 phoenix-load-balancer/pom.xml                   |  2 +-
 phoenix-parcel/pom.xml                          |  2 +-
 phoenix-pherf/pom.xml                           |  2 +-
 phoenix-pig/pom.xml                             |  2 +-
 phoenix-queryserver-client/pom.xml              |  2 +-
 phoenix-queryserver/pom.xml                     |  2 +-
 phoenix-server/pom.xml                          |  2 +-
 phoenix-spark/pom.xml                           |  2 +-
 phoenix-tracing-webapp/pom.xml                  |  2 +-
 pom.xml                                         |  4 +--
 17 files changed, 49 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 55a9a6e..14225ee 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index 2454de6..e211008 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-client</artifactId>
   <name>Phoenix Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index e1f8e2a..2d837a2 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
index 4fdddf5..d1f05f8 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
@@ -124,6 +124,36 @@ public class PhoenixRpcScheduler extends RpcScheduler {
     public void setMetadataExecutorForTesting(RpcExecutor executor) {
         this.metadataCallExecutor = executor;
     }
-    
-    
+
+    @Override
+    public int getReadQueueLength() {
+        return delegate.getReadQueueLength();
+    }
+
+    @Override
+    public int getWriteQueueLength() {
+        return delegate.getWriteQueueLength();
+    }
+
+    @Override
+    public int getScanQueueLength() {
+        return delegate.getScanQueueLength();
+    }
+
+    @Override
+    public int getActiveReadRpcHandlerCount() {
+        return delegate.getActiveReadRpcHandlerCount();
+    }
+
+    @Override
+    public int getActiveWriteRpcHandlerCount() {
+        return delegate.getActiveWriteRpcHandlerCount();
+    }
+
+    @Override
+    public int getActiveScanRpcHandlerCount() {
+        return delegate.getActiveScanRpcHandlerCount();
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index d61a9aa..8a78010 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-hive/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
index 809fbea..804ba5f 100644
--- a/phoenix-hive/pom.xml
+++ b/phoenix-hive/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-hive</artifactId>
   <name>Phoenix - Hive</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-kafka/pom.xml b/phoenix-kafka/pom.xml
index af6b4fe..08a8bfd 100644
--- a/phoenix-kafka/pom.xml
+++ b/phoenix-kafka/pom.xml
@@ -26,7 +26,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+		<version>4.14.0-cdh5.12.2-SNAPSHOT</version>
 	</parent>
 	<artifactId>phoenix-kafka</artifactId>
 	<name>Phoenix - Kafka</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-load-balancer/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-load-balancer/pom.xml b/phoenix-load-balancer/pom.xml
index 81e124a..cdb4c1b 100644
--- a/phoenix-load-balancer/pom.xml
+++ b/phoenix-load-balancer/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-load-balancer</artifactId>
   <name>Phoenix Load Balancer</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-parcel/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-parcel/pom.xml b/phoenix-parcel/pom.xml
index 5e6fccc..8e7b096 100644
--- a/phoenix-parcel/pom.xml
+++ b/phoenix-parcel/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-parcel</artifactId>
   <name>Phoenix Parcels for CDH</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 7831f35..0634a01 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+		<version>4.14.0-cdh5.12.2-SNAPSHOT</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index e5d0d52..cd4f6bc 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index 83cfde6..86c56b9 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-queryserver/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index 7180b18..e6c32cb 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 01a7bc3..d33bdaa 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index a45d4b5..b3b6e1c 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/phoenix-tracing-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index 93edc43..5226f84 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+      <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15713968/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 970428a..947ed59 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+  <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>
@@ -86,7 +86,7 @@
   <parent>
     <groupId>com.cloudera.cdh</groupId>
     <artifactId>cdh-root</artifactId>
-    <version>5.11.2</version>
+    <version>5.12.2</version>
   </parent>
 
   <scm>


[09/21] phoenix git commit: PHOENIX-4658 IllegalStateException: requestSeek cannot be called on ReversedKeyValueHeap (Toshihiro Suzuki)

Posted by pb...@apache.org.
PHOENIX-4658 IllegalStateException: requestSeek cannot be called on ReversedKeyValueHeap (Toshihiro Suzuki)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 0987d09fafda519bff73e50f7dcd4fbd303678ec
Parents: 08564a9
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Apr 11 21:31:38 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:27:22 2018 +0100

----------------------------------------------------------------------
 .../phoenix/end2end/MultiCfQueryExecIT.java     | 47 ++++++++++++++++++++
 .../apache/phoenix/compile/OrderByCompiler.java | 11 +++--
 .../apache/phoenix/compile/QueryCompiler.java   | 12 +----
 .../java/org/apache/phoenix/parse/HintNode.java |  4 ++
 .../java/org/apache/phoenix/util/ScanUtil.java  |  2 +
 5 files changed, 62 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0987d09f/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
index d94df6c..01da2d8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
@@ -31,11 +31,13 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.List;
 import java.util.Properties;
 
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
@@ -406,4 +408,49 @@ public class MultiCfQueryExecIT extends ParallelStatsEnabledIT {
             assertFalse(rs.next());
         }
     }
+
+    @Test
+    public void testBug4658() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+          Statement stmt = conn.createStatement()) {
+            String tableName = generateUniqueName();
+
+            stmt.execute("CREATE TABLE " + tableName + " ("
+                + "COL1 VARCHAR NOT NULL,"
+                + "COL2 VARCHAR NOT NULL,"
+                + "COL3 VARCHAR,"
+                + "FAM.COL4 VARCHAR,"
+                + "CONSTRAINT TRADE_EVENT_PK PRIMARY KEY (COL1, COL2))");
+            stmt.execute("UPSERT INTO " + tableName + " (COL1, COL2) values ('111', 'AAA')");
+            stmt.execute("UPSERT INTO " + tableName + " (COL1, COL2) values ('222', 'AAA')");
+            conn.commit();
+
+            try (ResultSet rs = stmt.executeQuery(
+              "SELECT * FROM " + tableName + " WHERE COL2 = 'AAA' ORDER BY COL1 DESC")) {
+                assertTrue(rs.next());
+                assertEquals(rs.getString("COL1"), "222");
+                assertEquals(rs.getString("COL2"), "AAA");
+                assertTrue(rs.next());
+                assertEquals(rs.getString("COL1"), "111");
+                assertEquals(rs.getString("COL2"), "AAA");
+                assertFalse(rs.next());
+            }
+
+            // Tests for FORWARD_SCAN hint
+            String query = "SELECT /*+ FORWARD_SCAN */ * FROM " + tableName + " WHERE COL2 = 'AAA' ORDER BY COL1 DESC";
+            try (ResultSet rs = stmt.executeQuery("EXPLAIN " + query)) {
+                String explainPlan = QueryUtil.getExplainPlan(rs);
+                assertFalse(explainPlan.contains("REVERSE"));
+            }
+            try (ResultSet rs = stmt.executeQuery(query)) {
+                assertTrue(rs.next());
+                assertEquals(rs.getString("COL1"), "222");
+                assertEquals(rs.getString("COL2"), "AAA");
+                assertTrue(rs.next());
+                assertEquals(rs.getString("COL1"), "111");
+                assertEquals(rs.getString("COL2"), "AAA");
+                assertFalse(rs.next());
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0987d09f/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
index 1097f70..b83c7a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
@@ -30,6 +30,7 @@ import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.parse.OrderByNode;
 import org.apache.phoenix.parse.ParseNode;
@@ -154,12 +155,16 @@ public class OrderByCompiler {
         // If we're ordering by the order returned by the scan, we don't need an order by
         if (isInRowKeyOrder && tracker.isOrderPreserving()) {
             if (tracker.isReverse()) {
-                // Don't use reverse scan if we're using a skip scan, as our skip scan doesn't support this yet.
+                // Don't use reverse scan if:
+                // 1) we're using a skip scan, as our skip scan doesn't support this yet.
+                // 2) we have the FORWARD_SCAN hint set to choose to keep loading of column
+                //    families on demand versus doing a reverse scan
                 // REV_ROW_KEY_ORDER_BY scan would not take effect for a projected table, so don't return it for such table types.
                 if (context.getConnection().getQueryServices().getProps().getBoolean(QueryServices.USE_REVERSE_SCAN_ATTRIB, QueryServicesOptions.DEFAULT_USE_REVERSE_SCAN)
                         && !context.getScanRanges().useSkipScanFilter()
                         && context.getCurrentTable().getTable().getType() != PTableType.PROJECTED
-                        && context.getCurrentTable().getTable().getType() != PTableType.SUBQUERY) {
+                        && context.getCurrentTable().getTable().getType() != PTableType.SUBQUERY
+                        && !statement.getHint().hasHint(Hint.FORWARD_SCAN)) {
                     return OrderBy.REV_ROW_KEY_ORDER_BY;
                 }
             } else {
@@ -172,4 +177,4 @@ public class OrderByCompiler {
 
     private OrderByCompiler() {
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0987d09f/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 9568ad8..3e5f5ee 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -66,7 +66,6 @@ import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.SubqueryParseNode;
 import org.apache.phoenix.parse.TableNode;
 import org.apache.phoenix.query.ConnectionQueryServices;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.AmbiguousColumnException;
@@ -91,13 +90,6 @@ import com.google.common.collect.Sets;
  */
 public class QueryCompiler {
     private static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
-    /*
-     * Not using Scan.setLoadColumnFamiliesOnDemand(true) because we don't
-     * want to introduce a dependency on 0.94.5 (where this feature was
-     * introduced). This will do the same thing. Once we do have a
-     * dependency on 0.94.5 or above, switch this around.
-     */
-    private static final String LOAD_COLUMN_FAMILIES_ON_DEMAND_ATTR = "_ondemand_";
     private final PhoenixStatement statement;
     private final Scan scan;
     private final Scan originalScan;
@@ -128,9 +120,7 @@ public class QueryCompiler {
         this.noChildParentJoinOptimization = select.getHint().hasHint(Hint.NO_CHILD_PARENT_JOIN_OPTIMIZATION);
         ConnectionQueryServices services = statement.getConnection().getQueryServices();
         this.costBased = services.getProps().getBoolean(QueryServices.COST_BASED_OPTIMIZER_ENABLED, QueryServicesOptions.DEFAULT_COST_BASED_OPTIMIZER_ENABLED);
-        if (services.getLowestClusterHBaseVersion() >= PhoenixDatabaseMetaData.ESSENTIAL_FAMILY_VERSION_THRESHOLD) {
-            this.scan.setAttribute(LOAD_COLUMN_FAMILIES_ON_DEMAND_ATTR, QueryConstants.TRUE);
-        }
+        scan.setLoadColumnFamiliesOnDemand(true);
         if (select.getHint().hasHint(Hint.NO_CACHE)) {
             scan.setCacheBlocks(false);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0987d09f/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
index 6d8451b..39e9b05 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
@@ -104,6 +104,10 @@ public class HintNode {
       * Enforces a serial scan.
       */
      SERIAL,
+        /**
+         * Enforces a forward scan.
+         */
+        FORWARD_SCAN,
     };
 
     private final Map<Hint,String> hints;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0987d09f/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 9c710c1..dd885fd 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
@@ -612,10 +612,12 @@ public class ScanUtil {
     
     public static void setReversed(Scan scan) {
         scan.setAttribute(BaseScannerRegionObserver.REVERSE_SCAN, PDataType.TRUE_BYTES);
+        scan.setLoadColumnFamiliesOnDemand(false);
     }
 
     public static void unsetReversed(Scan scan) {
         scan.setAttribute(BaseScannerRegionObserver.REVERSE_SCAN, PDataType.FALSE_BYTES);
+        scan.setLoadColumnFamiliesOnDemand(true);
     }
 
     private static byte[] getReversedRow(byte[] startRow) {


[19/21] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

Posted by pb...@apache.org.
PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 40226499bfab76e2920b0fc1160c25031700596f
Parents: 8cda814
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Apr 13 22:19:15 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:31:40 2018 +0100

----------------------------------------------------------------------
 .../phoenix/query/ConnectionQueryServicesImpl.java       | 11 -----------
 1 file changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/40226499/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 fa5d7e2..7f97c74 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
@@ -1130,10 +1130,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
-                // Remove the splitPolicy attribute to prevent HBASE-12570
-                if (isMetaTable) {
-                    newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-                }
                 try {
                     if (splits == null) {
                         admin.createTable(newDesc);
@@ -1150,13 +1146,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
                 if (isMetaTable && !isUpgradeRequired()) {
                     checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
-                    /*
-                     * Now we modify the table to add the split policy, since we know that the client and
-                     * server and compatible. This works around HBASE-12570 which causes the cluster to be
-                     * brought down.
-                     */
-                    newDesc.setValue(HTableDescriptor.SPLIT_POLICY, MetaDataSplitPolicy.class.getName());
-                    modifyTable(physicalTableName, newDesc, true);
                 }
                 return null;
             } else {


[16/21] phoenix git commit: PHOENIX-4496 Fix RowValueConstructorIT and IndexMetadataIT

Posted by pb...@apache.org.
PHOENIX-4496 Fix RowValueConstructorIT and IndexMetadataIT


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

Branch: refs/heads/4.x-cdh5.12
Commit: f71654a5c68e403d6c9c9fd3d535ee9cfc1b73f7
Parents: 8eaca12
Author: Ankit Singhal <an...@gmail.com>
Authored: Fri Apr 13 10:54:32 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:31:12 2018 +0100

----------------------------------------------------------------------
 .../phoenix/hbase/index/scanner/ScannerBuilder.java | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f71654a5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
index ad09c0c..703fcd2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FamilyFilter;
 import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -98,6 +99,21 @@ public class ScannerBuilder {
       }
       columnFilters.addFilter(columnFilter);
     }
+    
+    if(columns.isEmpty()){
+        columnFilters.addFilter(new FilterBase() {
+            
+            @Override
+            public boolean filterAllRemaining() throws IOException {
+                return true;
+            }
+
+            @Override
+            public ReturnCode filterKeyValue(Cell v) throws IOException {
+                return Filter.ReturnCode.INCLUDE;
+            }
+        });
+    }
     return columnFilters;
   }
 


[06/21] phoenix git commit: PHOENIX-4672 Align the kerberos principal config keys

Posted by pb...@apache.org.
PHOENIX-4672 Align the kerberos principal config keys

Fix the configuration keys in a manner that won't break users
who are using the 'old' name.


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

Branch: refs/heads/4.x-cdh5.12
Commit: 79c25f67ba0cb4ce9047b419123b1dfc142cf403
Parents: 6e899e5
Author: Josh Elser <el...@apache.org>
Authored: Tue Apr 10 19:08:57 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:26:37 2018 +0100

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/query/QueryServices.java    | 3 ++-
 .../java/org/apache/phoenix/queryserver/server/QueryServer.java  | 4 ++++
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/79c25f67/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 43b9e5a..21f043c 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
@@ -237,7 +237,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_KEYTAB_FILENAME_ATTRIB = "phoenix.queryserver.keytab.file";
     public static final String QUERY_SERVER_HTTP_KEYTAB_FILENAME_ATTRIB = "phoenix.queryserver.http.keytab.file";
     public static final String QUERY_SERVER_KERBEROS_PRINCIPAL_ATTRIB = "phoenix.queryserver.kerberos.principal";
-    public static final String QUERY_SERVER_KERBEROS_HTTP_PRINCIPAL_ATTRIB = "phoenix.queryserver.kerberos.http.principal";
+    public static final String QUERY_SERVER_KERBEROS_HTTP_PRINCIPAL_ATTRIB_LEGACY = "phoenix.queryserver.kerberos.http.principal";
+    public static final String QUERY_SERVER_KERBEROS_HTTP_PRINCIPAL_ATTRIB = "phoenix.queryserver.http.kerberos.principal";
     public static final String QUERY_SERVER_DNS_NAMESERVER_ATTRIB = "phoenix.queryserver.dns.nameserver";
     public static final String QUERY_SERVER_DNS_INTERFACE_ATTRIB = "phoenix.queryserver.dns.interface";
     public static final String QUERY_SERVER_HBASE_SECURITY_CONF_ATTRIB = "hbase.security.authentication";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/79c25f67/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
index 6b1fcfe..8436086 100644
--- a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
@@ -246,6 +246,10 @@ public final class QueryServer extends Configured implements Tool, Runnable {
         File keytab = new File(keytabPath);
         String httpKeytabPath = getConf().get(QueryServices.QUERY_SERVER_HTTP_KEYTAB_FILENAME_ATTRIB, null);
         String httpPrincipal = getConf().get(QueryServices.QUERY_SERVER_KERBEROS_HTTP_PRINCIPAL_ATTRIB, null);
+        // Backwards compat for a configuration key change
+        if (httpPrincipal == null) {
+          httpPrincipal = getConf().get(QueryServices.QUERY_SERVER_KERBEROS_HTTP_PRINCIPAL_ATTRIB_LEGACY, null);
+        }
         File httpKeytab = null;
         if (null != httpKeytabPath)
           httpKeytab = new File(httpKeytabPath);


[02/21] phoenix git commit: PHOENIX-4669 NoSuchColumnFamilyException when creating index on views that are built on tables which have named column family

Posted by pb...@apache.org.
PHOENIX-4669 NoSuchColumnFamilyException when creating index on views that are built on tables which have named column family

Signed-off-by: ss77892 <ss...@apache.org>


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

Branch: refs/heads/4.x-cdh5.12
Commit: 4d9cc9294733161ebc915042e4c173f74906fad0
Parents: 7871e72
Author: Toshihiro Suzuki <br...@gmail.com>
Authored: Thu Mar 29 09:17:37 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:24:59 2018 +0100

----------------------------------------------------------------------
 .../phoenix/end2end/index/ViewIndexIT.java      | 45 +++++++++++++++++++-
 .../query/ConnectionQueryServicesImpl.java      | 27 ++++++------
 2 files changed, 57 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4d9cc929/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index 53bb550..8ffd798 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -31,6 +31,7 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -443,5 +444,47 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
         assertFalse(rs.next());
         assertEquals(indexName, stmt.getQueryPlan().getContext().getCurrentTable().getTable().getName().getString());
     }
-    
+
+    @Test
+    public void testCreatingIndexOnViewBuiltOnTableWithOnlyNamedColumnFamilies() throws Exception {
+        try (Connection c = getConnection(); Statement s = c.createStatement()) {
+            String tableName = generateUniqueName();
+            String viewName = generateUniqueName();
+            String indexName = generateUniqueName();
+
+            c.setAutoCommit(true);
+            s.execute("CREATE TABLE " + tableName + " (COL1 VARCHAR PRIMARY KEY, CF.COL2 VARCHAR)");
+            s.executeUpdate("UPSERT INTO " + tableName + " VALUES ('AAA', 'BBB')");
+            s.execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName);
+            s.execute("CREATE INDEX " + indexName + " ON " + viewName + " (CF.COL2)");
+
+            try (ResultSet rs = s.executeQuery("SELECT * FROM " + viewName + " WHERE CF.COL2 = 'BBB'")) {
+                assertTrue(rs.next());
+                assertEquals("AAA", rs.getString("COL1"));
+                assertEquals("BBB", rs.getString("COL2"));
+            }
+        }
+        try (Connection c = getConnection(); Statement s = c.createStatement()) {
+            String tableName = generateUniqueName();
+            String viewName = generateUniqueName();
+            String index1Name = generateUniqueName();
+            String index2Name = generateUniqueName();
+
+            c.setAutoCommit(true);
+            s.execute("create table " + tableName + " (i1 integer primary key, c2.i2 integer, c3.i3 integer, c4.i4 integer)");
+            s.execute("create view " + viewName + " as select * from " + tableName + " where c2.i2 = 1");
+            s.executeUpdate("upsert into " + viewName + "(i1, c3.i3, c4.i4) VALUES (1, 1, 1)");
+            s.execute("create index " + index1Name + " ON " + viewName + " (c3.i3)");
+            s.execute("create index " + index2Name + " ON " + viewName + " (c3.i3) include (c4.i4)");
+            s.executeUpdate("upsert into " + viewName + "(i1, c3.i3, c4.i4) VALUES (2, 2, 2)");
+
+            try (ResultSet rs = s.executeQuery("select * from " + viewName + " WHERE c3.i3 = 1")) {
+                assertTrue(rs.next());
+                assertEquals(1, rs.getInt("i1"));
+                assertEquals(1, rs.getInt("i2"));
+                assertEquals(1, rs.getInt("i3"));
+                assertEquals(1, rs.getInt("i4"));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4d9cc929/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 eff406d..6df2f80 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
@@ -1690,22 +1690,20 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
-        Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
-        List<Pair<byte[],Map<String,Object>>> families = Lists.newArrayListWithExpectedSize(Math.max(1, table.getColumnFamilies().size()+1));
-        if (families.isEmpty()) {
-            byte[] familyName = SchemaUtil.getEmptyColumnFamily(table);
+        List<Pair<byte[],Map<String,Object>>> families = Lists.newArrayListWithExpectedSize(Math.max(1, table.getColumnFamilies().size() + 1));
+
+        // Create all column families that the parent table has
+        for (PColumnFamily family : table.getColumnFamilies()) {
+            byte[] familyName = family.getName().getBytes();
             Map<String,Object> familyProps = createPropertiesMap(htableDesc.getFamily(familyName).getValues());
-            families.add(new Pair<byte[],Map<String,Object>>(familyName, familyProps));
-        } else {
-            for (PColumnFamily family : table.getColumnFamilies()) {
-                byte[] familyName = family.getName().getBytes();
-                Map<String,Object> familyProps = createPropertiesMap(htableDesc.getFamily(familyName).getValues());
-                families.add(new Pair<byte[],Map<String,Object>>(familyName, familyProps));
-            }
-            // Always create default column family, because we don't know in advance if we'll
-            // need it for an index with no covered columns.
-            families.add(new Pair<byte[],Map<String,Object>>(table.getDefaultFamilyName().getBytes(), Collections.<String,Object>emptyMap()));
+            families.add(new Pair<>(familyName, familyProps));
         }
+        // Always create default column family, because we don't know in advance if we'll
+        // need it for an index with no covered columns.
+        byte[] defaultFamilyName = table.getDefaultFamilyName() == null ?
+          QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : table.getDefaultFamilyName().getBytes();
+        families.add(new Pair<>(defaultFamilyName, Collections.<String,Object>emptyMap()));
+
         byte[][] splits = null;
         if (table.getBucketNum() != null) {
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
@@ -1713,6 +1711,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
         // Transfer over table values into tableProps
         // TODO: encapsulate better
+        Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
         tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
         tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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/8eaca121/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/8eaca121/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 15d8ac3..2f41dc3 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/8eaca121/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/8eaca121/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 c5233d3..d33e3fe 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/8eaca121/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 5e6f756..949e6ed 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/8eaca121/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 ba6a08f..cc7221e 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/8eaca121/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 9caf7fb..add0628 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
@@ -47,6 +47,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;
@@ -297,6 +298,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/8eaca121/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 f526419..25b9fb0 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/8eaca121/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 7c154f0..f4ecac2 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/8eaca121/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 0b72ada..b75119b 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
@@ -48,6 +48,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 {
@@ -152,4 +154,6 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public User getUser();
 
     public QueryLoggerDisruptor getQueryDisruptor();
-}
\ No newline at end of file
+
+    public PhoenixTransactionClient initTransactionClient(TransactionFactory.Provider provider);
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 6627a84..5cb14d6 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
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 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.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
@@ -138,10 +139,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;
@@ -226,8 +227,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;
@@ -264,7 +267,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;
@@ -306,6 +309,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.
@@ -410,23 +414,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();
@@ -517,7 +509,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) {
@@ -858,9 +854,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.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
@@ -923,13 +920,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
 
             if (isTransactional) {
-                if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
-                    descriptor.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 (!descriptor.hasCoprocessor(coprocessorClass.getName())) {
+                    descriptor.addCoprocessor(coprocessorClass.getName(), null, priority - 10, null);
                 }
             } else {
-                // If exception on alter table to transition back to non transactional
-                if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
-                    descriptor.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 && descriptor.hasCoprocessor(coprocessorClass.getName())) {
+                        descriptor.removeCoprocessor(coprocessorClass.getName());
+                    }
                 }
             }
         } catch (IOException e) {
@@ -1126,7 +1132,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();
@@ -2895,6 +2901,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());
+                }
             }
 
 
@@ -4080,7 +4094,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public String getUserName() {
         return userName;
     }
-    
+
     @Override
     public User getUser() {
         return user;
@@ -4515,4 +4529,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/8eaca121/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 ad354d1..aa8209d 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
@@ -82,7 +82,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;
@@ -116,7 +117,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     private final Configuration config;
 
     private User user;
-    
+
     public ConnectionlessQueryServicesImpl(QueryServices services, ConnectionInfo connInfo, Properties info) {
         super(services);
         userName = connInfo.getPrincipal();
@@ -141,7 +142,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);
     }
 
@@ -682,4 +682,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/8eaca121/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 f5c8a59..ed9b9da 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
@@ -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.Provider;
 
 
 public class DelegateConnectionQueryServices extends DelegateQueryServices implements ConnectionQueryServices {
@@ -363,6 +365,11 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public QueryLoggerDisruptor getQueryDisruptor() {
         return getDelegate().getQueryDisruptor();
     }
-    
-    
-}
\ No newline at end of file
+
+
+
+    @Override
+    public PhoenixTransactionClient initTransactionClient(Provider provider) {
+        return getDelegate().initTransactionClient(provider);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 ae12e01..d181fc8 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
@@ -100,6 +100,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.TYPE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
@@ -339,6 +340,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/8eaca121/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 21f043c..29d18d9 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
@@ -207,6 +207,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/8eaca121/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 58c9812..70ac11b 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;
 
@@ -263,6 +264,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/8eaca121/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();
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 d252879..1fb668e 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
@@ -83,6 +83,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;
@@ -222,6 +223,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;
@@ -280,7 +284,7 @@ public class MetaDataClient {
                     INDEX_TYPE + "," +
                     STORE_NULLS + "," +
                     BASE_COLUMN_COUNT + "," +
-                    TRANSACTIONAL + "," +
+                    TRANSACTION_PROVIDER + "," +
                     UPDATE_CACHE_FREQUENCY + "," +
                     IS_NAMESPACE_MAPPED + "," +
                     AUTO_PARTITION_SEQ +  "," +
@@ -572,14 +576,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
@@ -632,16 +633,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)) {
@@ -1236,8 +1241,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);
@@ -1861,7 +1866,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;
@@ -1877,7 +1882,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();
@@ -2018,31 +2023,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();
@@ -2050,8 +2069,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(HColumnDescriptor.TTL);
                 if (ttl != null) {
@@ -2063,7 +2083,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
@@ -2098,7 +2118,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) {
@@ -2481,7 +2501,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);
             }
             
@@ -2621,7 +2641,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) {
@@ -2746,7 +2770,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;
@@ -3251,8 +3275,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<>();
@@ -4247,6 +4271,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)) {
@@ -4369,6 +4395,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);
             }
@@ -4381,6 +4423,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;
@@ -4421,6 +4464,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;
         }
@@ -4490,6 +4541,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;
@@ -4570,5 +4622,14 @@ 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/8eaca121/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();


[20/21] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

Posted by pb...@apache.org.
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 9fc11de1586c93c53d2fe994e374e2f15b3b3f05
Parents: 4022649
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Apr 13 22:30:00 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:31:51 2018 +0100

----------------------------------------------------------------------
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java      | 41 ++++++++++++++------
 .../query/ConnectionlessQueryServicesImpl.java  | 35 +++++++++++++++--
 .../apache/phoenix/query/QueryConstants.java    | 14 +++----
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java     |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  4 +-
 7 files changed, 74 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 26f8198..36d6f0d 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
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
     public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-    public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-    public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-    public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-    public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
     public static final int DEFAULT_LOG_VERSIONS = 10;
     public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 7f97c74..02fcf24 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
@@ -720,10 +720,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private HColumnDescriptor generateColumnFamilyDescriptor(Pair<byte[],Map<String,Object>> family, PTableType tableType) throws SQLException {
         HColumnDescriptor columnDesc = new HColumnDescriptor(family.getFirst());
         if (tableType != PTableType.VIEW) {
-            if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
-                columnDesc.setKeepDeletedCells(props.getBoolean(
-                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-            }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             columnDesc.setBloomFilterType(BloomType.NONE);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -2459,8 +2455,29 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     // Available for testing
-    protected String getSystemCatalogDML() {
-        return QueryConstants.CREATE_TABLE_METADATA;
+    protected String getSystemCatalogTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+    }
+
+    // Available for testing
+    protected String getFunctionTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+    }
+
+    // Available for testing
+    protected String getLogTableDDL() {
+        return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+    }
+
+    private String setSystemDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+    }
+
+    private String setSystemLogDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
     }
 
     @Override
@@ -2503,7 +2520,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                          scnProps, newEmptyMetaData())) {
                                 try {
                                     metaConnection.setRunningUpgrade(true);
-                                    metaConnection.createStatement().executeUpdate(getSystemCatalogDML());
+                                    metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
                                 } catch (NewerTableAlreadyExistsException ignore) {
                                     // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed
                                     // timestamp. A TableAlreadyExistsException is not thrown, since the table only exists
@@ -2665,10 +2682,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             metaConnection.createStatement().execute(QueryConstants.CREATE_STATS_TABLE_METADATA);
         } catch (TableAlreadyExistsException ignore) {}
         try {
-            metaConnection.createStatement().execute(QueryConstants.CREATE_FUNCTION_METADATA);
+            metaConnection.createStatement().execute(getFunctionTableDDL());
         } catch (TableAlreadyExistsException ignore) {}
         try {
-            metaConnection.createStatement().execute(QueryConstants.CREATE_LOG_METADATA);
+            metaConnection.createStatement().execute(getLogTableDDL());
         } catch (TableAlreadyExistsException ignore) {}
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
@@ -2971,7 +2988,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 createSysMutexTableIfNotExists(admin);
             }
             try {
-                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+                metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
             } catch (NewerTableAlreadyExistsException ignore) {
                 // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed
                 // timestamp. A TableAlreadyExistsException is not thrown, since the table only exists
@@ -3093,10 +3110,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             try {
-                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
+                metaConnection.createStatement().executeUpdate(getFunctionTableDDL());
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
             try {
-                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
+                metaConnection.createStatement().executeUpdate(getLogTableDDL());
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
 
             // In case namespace mapping is enabled and system table to system namespace mapping is also enabled,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 14abd63..c6c2617 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,11 +84,13 @@ 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;
+import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.JDBCUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SequenceUtil;
 
@@ -106,6 +108,7 @@ import com.google.common.collect.Maps;
 public class ConnectionlessQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices  {
     private static ServerName SERVER_NAME = ServerName.parseServerName(HConstants.LOCALHOST + Addressing.HOSTNAME_PORT_SEPARATOR + HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
     
+    private final ReadOnlyProps props;
     private PMetaData metaData;
     private final Map<SequenceKey, SequenceInfo> sequenceMap = Maps.newHashMap();
     private final String userName;
@@ -143,12 +146,38 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         // on the server side during testing.
         this.config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration(config);
         this.guidePostsCache = new GuidePostsCache(this, config);
+        // set replication required parameter
+        ConfigUtil.setReplicationConfigIfAbsent(this.config);
+        this.props = new ReadOnlyProps(this.config.iterator());
     }
 
     private PMetaData newEmptyMetaData() {
         return new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps());
     }
 
+    protected String getSystemCatalogTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+    }
+
+    protected String getFunctionTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+    }
+
+    protected String getLogTableDDL() {
+        return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+    }
+
+    private String setSystemDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+    }
+
+    private String setSystemLogDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+    }
+
     @Override
     public ConnectionQueryServices getChildQueryServices(ImmutableBytesWritable childId) {
         return this; // Just reuse the same query services
@@ -307,7 +336,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                 metaConnection = new PhoenixConnection(this, globalUrl, scnProps, newEmptyMetaData());
                 metaConnection.setRunningUpgrade(true);
                 try {
-                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+                    metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
                 } catch (TableAlreadyExistsException ignore) {
                     // Ignore, as this will happen if the SYSTEM.TABLE already exists at this fixed timestamp.
                     // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
@@ -330,11 +359,11 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                 }
                 
                 try {
-                   metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
+                    metaConnection.createStatement().executeUpdate(getFunctionTableDDL());
                 } catch (NewerTableAlreadyExistsException ignore) {
                 }
                 try {
-                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
+                    metaConnection.createStatement().executeUpdate(getLogTableDDL());
                 } catch (NewerTableAlreadyExistsException ignore) {}
             } catch (SQLException e) {
                 sqlE = e;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 d181fc8..22fa2f4 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
@@ -343,8 +343,8 @@ public interface QueryConstants {
             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" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n" +
+            HConstants.VERSIONS + "=%s,\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
@@ -361,8 +361,6 @@ public interface QueryConstants {
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY ("
             + PHYSICAL_NAME + ","
             + COLUMN_FAMILY + ","+ GUIDE_POST_KEY+"))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_STAT_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_STATS_KEEP_DELETED_CELLS + ",\n" +
             // Install split policy to prevent a physical table's stats from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
@@ -382,8 +380,6 @@ public interface QueryConstants {
             CYCLE_FLAG + " BOOLEAN, \n" +
             LIMIT_REACHED_FLAG + " BOOLEAN \n" +
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + "," + SEQUENCE_SCHEMA + "," + SEQUENCE_NAME + "))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n" +
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
     public static final String CREATE_SYSTEM_SCHEMA = "CREATE SCHEMA " + SYSTEM_CATALOG_SCHEMA;
     public static final String UPGRADE_TABLE_SNAPSHOT_PREFIX = "_UPGRADING_TABLE_";
@@ -407,8 +403,8 @@ public interface QueryConstants {
             MIN_VALUE + " VARCHAR, \n" +
             MAX_VALUE + " VARCHAR, \n" +
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ", " + FUNCTION_NAME + ", " + TYPE + ", " + ARG_POSITION + "))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n"+
+            HConstants.VERSIONS + "=%s,\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n"+
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
@@ -434,7 +430,7 @@ public interface QueryConstants {
             SCAN_METRICS_JSON + " VARCHAR, \n" +
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (QUERY_ID))\n" +
             HConstants.VERSIONS + "= " + MetaDataProtocol.DEFAULT_LOG_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n"+
+            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n"+
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE+ ",\n" + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 29d18d9..8cc156c 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
@@ -204,7 +204,6 @@ public interface QueryServices extends SQLCloseable {
     public static final String ALLOW_ONLINE_TABLE_SCHEMA_UPDATE = "hbase.online.schema.update.enable";
     public static final String NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK = "phoenix.schema.change.retries";
     public static final String DELAY_FOR_SCHEMA_UPDATE_CHECK = "phoenix.schema.change.delay";
-    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";
@@ -252,6 +251,10 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "phoenix.queryserver.remoteUserExtractor.param";
     public static final String QUERY_SERVER_DISABLE_KERBEROS_LOGIN = "phoenix.queryserver.disable.kerberos.login";
 
+    // metadata configs
+    public static final String DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB = "phoenix.system.default.keep.deleted.cells";
+    public static final String DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB = "phoenix.system.default.max.versions";
+
     public static final String RENEW_LEASE_ENABLED = "phoenix.scanner.lease.renew.enabled";
     public static final String RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS = "phoenix.scanner.lease.renew.interval";
     public static final String RENEW_LEASE_THRESHOLD_MILLISECONDS = "phoenix.scanner.lease.threshold";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 70ac11b..23aed7c 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
@@ -213,6 +213,8 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_ALLOW_LOCAL_INDEX = true;
     public static final int DEFAULT_INDEX_HANDLER_COUNT = 30;
     public static final int DEFAULT_METADATA_HANDLER_COUNT = 30;
+    public static final int DEFAULT_SYSTEM_MAX_VERSIONS = 1;
+    public static final boolean DEFAULT_SYSTEM_KEEP_DELETED_CELLS = false;
 
     // Retries when doing server side writes to SYSTEM.CATALOG
     // 20 retries with 100 pause = 230 seconds total retry time
@@ -257,7 +259,6 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_ALLOW_ONLINE_TABLE_SCHEMA_UPDATE = true;
     public static final int DEFAULT_RETRIES_FOR_SCHEMA_UPDATE_CHECK = 10;
     public static final long DEFAULT_DELAY_FOR_SCHEMA_UPDATE_CHECK = 5 * 1000; // 5 seconds.
-    public static final boolean DEFAULT_KEEP_DELETED_CELLS = false;
     public static final boolean DEFAULT_STORE_NULLS = false;
 
     // TODO Change this to true as part of PHOENIX-1543

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index 741bdff..63f2183 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -223,7 +223,7 @@ public class UpgradeUtil {
 
         Scan scan = new Scan();
         scan.setRaw(true);
-        scan.setMaxVersions(MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS);
+        scan.setMaxVersions();
         ResultScanner scanner = null;
         HTableInterface source = null;
         HTableInterface target = null;
@@ -698,7 +698,7 @@ public class UpgradeUtil {
                 boolean success = false;
                 Scan scan = new Scan();
                 scan.setRaw(true);
-                scan.setMaxVersions(MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS);
+                scan.setMaxVersions();
                 HTableInterface seqTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                 try {
                     boolean committed = false;


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 a7b31e8..1a11427 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
@@ -69,6 +69,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -134,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;
@@ -227,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
@@ -270,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());
         }
 
@@ -280,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());
     }
 
@@ -290,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());
     }
     
@@ -300,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());
     }
 
@@ -310,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());
     }
 
@@ -320,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());
     }
 
@@ -330,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());
     }
     
@@ -352,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());
     }
 
@@ -363,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());
     }
 
@@ -374,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());
     }
 
@@ -383,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);
     }
 
@@ -397,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)
@@ -405,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);
     }
 
@@ -414,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);
     }
     
@@ -454,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);
@@ -486,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;
@@ -1279,7 +1280,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();
@@ -1352,7 +1359,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) {
@@ -1418,7 +1425,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()}));
       }
@@ -1473,8 +1482,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/8eaca121/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/8eaca121/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 110868e..543eda1 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,17 +17,11 @@
  */
 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.BaseRegionObserver;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.hadoop.hbase.client.HTableInterface;
 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 {
 
@@ -56,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
 
     }
@@ -116,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 BaseRegionObserver 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 HTableInterface getTransactionalTable(HTableInterface 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/8eaca121/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 b0c1bfe..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.HTableInterface;
+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, HTableInterface 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/8eaca121/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 0662555..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
+++ /dev/null
@@ -1,363 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.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, HTableInterface 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 byte[] getTableName() {
-        // 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 setAutoFlush(boolean autoFlush) {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public boolean isAutoFlush() {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public long getWriteBufferSize() {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public void setWriteBufferSize(long writeBufferSize) throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public void flushCommits() throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public void close() throws IOException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public long incrementColumnValue(byte[] row, byte[] family,
-            byte[] qualifier, long amount, boolean writeToWAL)
-            throws IOException {
-        // TODO Auto-generated method stub
-        return 0;
-    }
-
-    @Override
-    public Boolean[] exists(List<Get> gets) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public void setAutoFlushTo(boolean autoFlush) {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @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 Object[] batch(List<? extends Row> actions) throws IOException,
-            InterruptedException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public <R> void batchCallback(List<? extends Row> actions,
-            Object[] results, Callback<R> callback) throws IOException,
-            InterruptedException {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public <R> Object[] batchCallback(List<? extends Row> actions,
-            Callback<R> callback) throws IOException, InterruptedException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @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
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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/8eaca121/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 52ff2f9..f3ad42f 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,20 +17,98 @@
  */
 package org.apache.phoenix.transaction;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
-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 java.io.IOException;
 import java.sql.SQLException;
 import java.util.concurrent.TimeoutException;
 
-public interface PhoenixTransactionContext {
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.phoenix.schema.PTable;
+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 HTableInterface getTransactionalTable(HTableInterface htable, boolean isImmutable) {
+            return null;
+        }
+    };
     /**
      * 
      * Visibility levels needed for checkpointing and  
@@ -49,22 +127,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
@@ -84,7 +146,7 @@ public interface PhoenixTransactionContext {
      * @throws SQLException
      */
     public void abort() throws SQLException;
-
+    
     /**
      * Create a checkpoint in a transaction as defined in [TEPHRA-96]
      * @throws SQLException
@@ -100,9 +162,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
      *
@@ -121,7 +191,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();
 
@@ -150,42 +221,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 BaseRegionObserver 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 HTableInterface getTransactionalTable(HTableInterface htable, boolean isImmutable);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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/8eaca121/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 {
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 7af1c08..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
+++ /dev/null
@@ -1,149 +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 org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-
-import java.io.IOException;
-import java.util.List;
-
-public interface PhoenixTransactionalTable extends HTableInterface {
-
-    /**
-     * Transaction version of {@link HTableInterface#get(Get get)}
-     * @param get
-     * @throws IOException
-     */
-    public Result get(Get get) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#put(Put put)}
-     * @param put
-     * @throws IOException
-     */
-    public void put(Put put) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#delete(Delete delete)}
-     *
-     * @param delete
-     * @throws IOException
-     */
-    public void delete(Delete delete) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#getScanner(Scan scan)}
-     *
-     * @param scan
-     * @return ResultScanner
-     * @throws IOException
-     */
-    public ResultScanner getScanner(Scan scan) throws IOException;
-
-    /**
-     * Returns Htable name
-     */
-    public byte[] getTableName();
-
-    /**
-     * Returns Htable configuration object
-     */
-    public Configuration getConfiguration();
-
-    /**
-     * Returns HTableDescriptor of Htable
-     * @throws IOException
-     */
-    public HTableDescriptor getTableDescriptor() throws IOException;
-
-    /**
-     * Checks if cell exists
-     * @throws IOException
-     */
-    public boolean exists(Get get) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#get(List gets)}
-     * @throws IOException
-     */
-    public Result[] get(List<Get> gets) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#getScanner(byte[] family)}
-     * @throws IOException
-     */
-    public ResultScanner getScanner(byte[] family) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#getScanner(byte[] family, byte[] qualifier)}
-     * @throws IOException
-     */
-    public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#put(List puts)}
-     * @throws IOException
-     */
-    public void put(List<Put> puts) throws IOException;
-
-    /**
-     * Transactional version of {@link HTableInterface#delete(List deletes)}
-     * @throws IOException
-     */
-    public void delete(List<Delete> deletes) throws IOException;
-
-    /**
-     * Delegates to {@link HTable#setAutoFlush(boolean autoFlush)}
-     */
-    public void setAutoFlush(boolean autoFlush);
-
-    /**
-     * Delegates to {@link HTable#isAutoFlush()}
-     */
-    public boolean isAutoFlush();
-
-    /**
-     * Delegates to see HTable.getWriteBufferSize()
-     */
-    public long getWriteBufferSize();
-
-    /**
-     * Delegates to see HTable.setWriteBufferSize()
-     */
-    public void setWriteBufferSize(long writeBufferSize) throws IOException;
-
-    /**
-     * Delegates to see HTable.flushCommits()
-     */
-    public void flushCommits() throws IOException;
-
-    /**
-     * Releases resources
-     * @throws IOException
-     */
-    public void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 77c3ab6..8b16210 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,71 +19,46 @@ 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.hbase.HConstants;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.conf.Configuration;
 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;
 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.Transaction.VisibilityLevel;
 import org.apache.tephra.TxConstants;
-import org.apache.tephra.distributed.PooledClientProvider;
-import org.apache.tephra.distributed.TransactionServiceClient;
-import org.apache.tephra.hbase.coprocessor.TransactionProcessor;
-import org.apache.tephra.inmemory.InMemoryTxSystemClient;
-import org.apache.tephra.util.TxUtils;
+import org.apache.tephra.hbase.TransactionAwareHTable;
 import org.apache.tephra.visibility.FenceWait;
 import org.apache.tephra.visibility.VisibilityFence;
-import org.apache.tephra.zookeeper.TephraZKClientService;
-import org.apache.tephra.distributed.TransactionService;
-import org.apache.tephra.metrics.TxMetricsCollector;
-import org.apache.tephra.persist.HDFSTransactionStateStorage;
-import org.apache.tephra.snapshot.SnapshotCodecProvider;
-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 com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Lists;
-import com.google.inject.util.Providers;
-
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class TephraTransactionContext implements PhoenixTransactionContext {
+import com.google.common.collect.Lists;
 
+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;
@@ -93,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();
@@ -117,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) {
@@ -181,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)
@@ -204,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()
@@ -249,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();
 
@@ -276,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);
 
@@ -300,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;
 
@@ -325,7 +271,6 @@ public class TephraTransactionContext implements PhoenixTransactionContext {
     public void reset() {
         tx = null;
         txAwares.clear();
-        this.e = null;
     }
 
     @Override
@@ -406,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 BaseRegionObserver 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
      */
@@ -511,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 HTableInterface getTransactionalTable(HTableInterface 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/8eaca121/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 795be9f..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.HTableInterface;
+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, HTableInterface 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;
     }
 
 }


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

Posted by pb...@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/8eaca121
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/8eaca121
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/8eaca121

Branch: refs/heads/4.x-cdh5.12
Commit: 8eaca12102c41a10694e9a343f97b8b83c15701c
Parents: 0535a17
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Apr 12 04:06:35 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:30:46 2018 +0100

----------------------------------------------------------------------
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java        |  34 +-
 .../phoenix/tx/FlappingTransactionIT.java       |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../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  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java     |   4 +-
 .../PhoenixTransactionalProcessor.java          |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java           |  29 ++
 .../UngroupedAggregateRegionObserver.java       |  10 +-
 .../coprocessor/generated/PTableProtos.java     | 110 +++++-
 .../phoenix/exception/SQLExceptionCode.java     |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 ++--
 .../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  |   6 +-
 .../query/ConnectionQueryServicesImpl.java      |  71 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |  11 +-
 .../query/DelegateConnectionQueryServices.java  |  13 +-
 .../apache/phoenix/query/QueryConstants.java    |   2 +
 .../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   |  68 ++--
 .../apache/phoenix/schema/TableProperty.java    |  18 +
 .../transaction/OmidTransactionContext.java     |  57 +--
 .../transaction/OmidTransactionProvider.java    |  54 +--
 .../transaction/OmidTransactionTable.java       | 363 -------------------
 .../transaction/PhoenixTransactionClient.java   |  23 ++
 .../transaction/PhoenixTransactionContext.java  | 169 +++++----
 .../transaction/PhoenixTransactionProvider.java |  51 +++
 .../transaction/PhoenixTransactionService.java  |  24 ++
 .../transaction/PhoenixTransactionalTable.java  | 149 --------
 .../transaction/TephraTransactionContext.java   | 205 +++--------
 .../transaction/TephraTransactionProvider.java  | 161 +++++++-
 .../transaction/TephraTransactionTable.java     | 350 ------------------
 .../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    |  93 ++++-
 .../phoenix/execute/CorrelatePlanTest.java      |   5 +-
 .../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, 1229 insertions(+), 1625 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 6b57148..237a8d2 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.HTableInterface;
 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);
             HTableInterface 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/8eaca121/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/8eaca121/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 200cf1c..0323c32 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;
@@ -225,9 +224,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);
+        HTableInterface txTable =
+                txContext.getTransactionalTable(htable, false);
 
         txContext.begin();
 
@@ -277,9 +276,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/8eaca121/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 fecfd9a..5421801 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
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 package org.apache.phoenix.tx;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 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.assertTrue;
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 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;
@@ -280,9 +280,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.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(index));
-        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getTableDescriptor().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");
@@ -357,7 +357,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         assertFalse(rs.next());
         
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
-        assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertFalse(htable.getTableDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
         assertEquals(1,conn.unwrap(PhoenixConnection.class).getQueryServices().
                 getTableDescriptor(Bytes.toBytes("SYSTEM." + nonTxTableName)).
                 getFamily(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES).getMaxVersions());
@@ -375,7 +375,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         PTable table = pconn.getTable(new PTableKey(null, t1));
         HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
         assertTrue(table.isTransactional());
-        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
         
         try {
             ddl = "ALTER TABLE " + t1 + " SET transactional=false";
@@ -409,7 +409,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.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(TephraTransactionalProcessor.class.getName()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 9286c2e..f1344e0 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;
 
@@ -44,6 +45,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;
@@ -129,6 +131,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();
@@ -139,6 +142,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/8eaca121/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 bf39dfe..42e2102 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.Test;
@@ -265,7 +266,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 + "");
@@ -329,7 +330,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/8eaca121/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/8eaca121/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 7985314..5f9c76c 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
@@ -33,7 +33,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;
@@ -783,8 +782,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/8eaca121/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 b5293bb..3faada7 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/8eaca121/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/8eaca121/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/8eaca121/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/8eaca121/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 1a9a686..22119a3 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/8eaca121/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 3310131..a426e39 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
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.ScannerContextUtil;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -49,10 +48,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;
 import org.apache.phoenix.util.TransactionUtil;
@@ -109,6 +106,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     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/8eaca121/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 05ad959..a2d008b 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;
@@ -228,6 +229,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;
@@ -295,6 +297,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     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);
@@ -326,6 +329,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             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 Coprocesso
     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);
@@ -961,7 +966,24 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         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];
@@ -1044,7 +1066,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         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/8eaca121/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 4c4c96f..a71ce0c 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,7 +95,7 @@ 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_SYSTEM_TABLE_TIMESTAMP_4_11_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0 = MIN_TABLE_TIMESTAMP + 28;
     // 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_4_14_0;
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 393a2f9..4968525 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
@@ -52,7 +52,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;
@@ -83,6 +82,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;
@@ -496,7 +496,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
 									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/8eaca121/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 0c26ecc..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixTransactionalProcessor.java
+++ /dev/null
@@ -1,28 +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 org.apache.phoenix.transaction.TransactionFactory;
-
-public class PhoenixTransactionalProcessor extends DelegateRegionObserver {
-
-    public PhoenixTransactionalProcessor() {
-        super(TransactionFactory.getTransactionProvider().getTransactionContext().getCoprocessor());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 448f61c..9d78659 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.CoprocessorService;
 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;
@@ -75,7 +75,7 @@ public class ServerCachingEndpointImpl extends ServerCachingService implements C
           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/8eaca121/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..a702bc4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TephraTransactionalProcessor.java
@@ -0,0 +1,29 @@
+/*
+ * 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 org.apache.tephra.hbase.coprocessor.TransactionProcessor;
+
+
+public class TephraTransactionalProcessor extends DelegateRegionObserver {
+
+    public TephraTransactionalProcessor() {
+        super(new TransactionProcessor());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 de57772..31b512a 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
@@ -277,7 +277,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);
            }
         }
     }
@@ -523,7 +523,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));
@@ -1023,7 +1023,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 {
@@ -1048,7 +1048,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);
@@ -1060,7 +1060,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/8eaca121/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/8eaca121/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 9cbc67e..e9e209b 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/8eaca121/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();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 f6d11a0..ca3e6d0 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
@@ -72,6 +72,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;
@@ -91,8 +93,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.KeyValueUtil;
@@ -139,7 +141,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;
@@ -181,17 +183,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);
         }
     }
 
@@ -234,7 +232,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
@@ -300,14 +298,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 HTableInterface getHTable(PTable table) throws SQLException {
         HTableInterface 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;
     }
@@ -334,13 +330,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
@@ -1069,6 +1084,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
@@ -1077,7 +1093,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();
@@ -1223,10 +1239,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();
@@ -1269,7 +1281,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);
                 }
@@ -1301,7 +1313,7 @@ public class MutationState implements SQLCloseable {
         numRows = 0;
         estimatedSize = 0;
         this.mutations.clear();
-        resetTransactionalState();
+        phoenixTransactionContext = PhoenixTransactionContext.NULL_CONTEXT;
     }
 
     private void resetTransactionalState() {
@@ -1365,13 +1377,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);
@@ -1476,9 +1493,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/8eaca121/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 7d6154e..6348d6d 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;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.client.Mutation;
 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.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -64,8 +63,6 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.types.PVarbinary;
 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.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -76,9 +73,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;
 
@@ -181,7 +175,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.


[08/21] phoenix git commit: PHOENIX-4653 Upgrading from namespace enabled cluster to latest version failing with UpgradeInProgressException(Rajeshbabu)

Posted by pb...@apache.org.
PHOENIX-4653 Upgrading from namespace enabled cluster to latest version failing with UpgradeInProgressException(Rajeshbabu)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 08564a99e4ea5c89441a55fe96a54122cc4a46c8
Parents: fe06c3b
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Wed Apr 11 13:48:04 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:27:12 2018 +0100

----------------------------------------------------------------------
 .../apache/phoenix/query/ConnectionQueryServicesImpl.java    | 8 +-------
 1 file changed, 1 insertion(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/08564a99/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 8c7441a..6627a84 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
@@ -2678,13 +2678,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                 sysCatalogTableName = e.getTable().getPhysicalName().getString();
                 if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
-                    // Ensure that the SYSTEM.MUTEX table has been created prior
-                    // to attempting to acquire the upgrade mutex. If namespace
-                    // mapping is enabled, we've already done this earlier in the
-                    // upgrade, so no need for a bunch of wasted RPCs.
-                    if (currentServerSideTableTimeStamp <= MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 &&
-                            !SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                    if (currentServerSideTableTimeStamp <= MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0) {
                         try (HBaseAdmin admin = getAdmin()) {
                             createSysMutexTableIfNotExists(admin, this.getProps());
                         }


[07/21] phoenix git commit: PHOENIX-2715 Query Log (addendum) Fix old com.lmax.disruptor dependency

Posted by pb...@apache.org.
PHOENIX-2715 Query Log (addendum)
Fix old com.lmax.disruptor dependency


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

Branch: refs/heads/4.x-cdh5.12
Commit: fe06c3b0717e484ada82a208968a9e773e4d0b98
Parents: 79c25f6
Author: Ankit Singhal <an...@gmail.com>
Authored: Wed Apr 11 13:07:08 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:26:50 2018 +0100

----------------------------------------------------------------------
 phoenix-core/pom.xml  | 7 -------
 phoenix-flume/pom.xml | 6 ------
 phoenix-kafka/pom.xml | 6 ------
 3 files changed, 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fe06c3b0/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 8fe8a10..e1f8e2a 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -345,12 +345,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>com.lmax</groupId>
-      <artifactId>disruptor</artifactId>
-      <version>3.2.0</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.htrace</groupId>
       <artifactId>htrace-core</artifactId>
     </dependency>
@@ -529,7 +523,6 @@
        <dependency>
         <groupId>com.lmax</groupId>
         <artifactId>disruptor</artifactId>
-        <version>${disruptor.version}</version>
       </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fe06c3b0/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index 0883e5e..d61a9aa 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -69,12 +69,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>com.lmax</groupId>
-      <artifactId>disruptor</artifactId>
-      <version>3.2.0</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.htrace</groupId>
       <artifactId>htrace-core</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fe06c3b0/phoenix-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-kafka/pom.xml b/phoenix-kafka/pom.xml
index 8718f5f..af6b4fe 100644
--- a/phoenix-kafka/pom.xml
+++ b/phoenix-kafka/pom.xml
@@ -143,12 +143,6 @@
 			<scope>test</scope>
 		</dependency>
 		<dependency>
-			<groupId>com.lmax</groupId>
-			<artifactId>disruptor</artifactId>
-			<version>3.2.0</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
 			<groupId>org.apache.htrace</groupId>
 			<artifactId>htrace-core</artifactId>
 		</dependency>


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

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 c191d8d..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,350 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.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.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-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, HTableInterface hTable) {
-        this(ctx, hTable, null);
-    }
-
-    public TephraTransactionTable(PhoenixTransactionContext ctx, HTableInterface 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 byte[] getTableName() {
-        return transactionAwareHTable.getTableName();
-    }
-
-    @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 setAutoFlush(boolean autoFlush) {
-        transactionAwareHTable.setAutoFlush(autoFlush);
-    }
-
-    @Override
-    public boolean isAutoFlush() {
-        return transactionAwareHTable.isAutoFlush();
-    }
-
-    @Override
-    public long getWriteBufferSize() {
-        return transactionAwareHTable.getWriteBufferSize();
-    }
-
-    @Override
-    public void setWriteBufferSize(long writeBufferSize) throws IOException {
-        transactionAwareHTable.setWriteBufferSize(writeBufferSize);
-    }
-
-    @Override
-    public void flushCommits() throws IOException {
-        transactionAwareHTable.flushCommits();
-    }
-
-    @Override
-    public void close() throws IOException {
-        transactionAwareHTable.close();
-    }
-
-    @Override
-    public long incrementColumnValue(byte[] row, byte[] family,
-            byte[] qualifier, long amount, boolean writeToWAL)
-            throws IOException {
-        return transactionAwareHTable.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
-    }
-
-    @Override
-    public Boolean[] exists(List<Get> gets) throws IOException {
-        return transactionAwareHTable.exists(gets);
-    }
-
-    @Override
-    public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
-        transactionAwareHTable.setAutoFlush(autoFlush, clearBufferOnFail);
-    }
-
-    @Override
-    public void setAutoFlushTo(boolean autoFlush) {
-        transactionAwareHTable.setAutoFlush(autoFlush);
-    }
-
-    @Override
-    public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
-        return transactionAwareHTable.getRowOrBefore(row, family);
-    }
-
-    @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 Object[] batch(List<? extends Row> actions) throws IOException,
-            InterruptedException {
-        return transactionAwareHTable.batch(actions);
-    }
-
-    @Override
-    public <R> void batchCallback(List<? extends Row> actions,
-            Object[] results, Callback<R> callback) throws IOException,
-            InterruptedException {
-        transactionAwareHTable.batchCallback(actions, results, callback);
-    }
-
-    @Override
-    public <R> Object[] batchCallback(List<? extends Row> actions,
-            Callback<R> callback) throws IOException, InterruptedException {
-        return transactionAwareHTable.batchCallback(actions, 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 i) {
-//        transactionAwareHTable.setOperationTimeout(i);
-    }
-
-    @Override
-    public int getOperationTimeout() {
-        return 0; //transactionAwareHTable.getOperationTimeout();
-    }
-
-    @Override
-    public void setRpcTimeout(int i) {
-//        transactionAwareHTable.setRpcTimeout(i);
-    }
-
-    @Override
-    public int getRpcTimeout() {
-        return 0; //transactionAwareHTable.getRpcTimeout();
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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/8eaca121/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 a5704f1..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.HTableInterface;
-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, HTableInterface 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/8eaca121/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 1c25c33..6cf6e56 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
@@ -88,7 +88,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;
@@ -1515,7 +1514,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/8eaca121/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 dd885fd..996e1dc 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.
@@ -930,5 +933,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));
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 ab76ffe..7d6dfd4 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
@@ -17,44 +17,65 @@
  */
 package org.apache.phoenix.util;
 
+import java.io.IOException;
 import java.sql.SQLException;
+import java.util.List;
+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.client.HTableInterface;
+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.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);
+    }
+    
+    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 PhoenixTransactionalTable getPhoenixTransactionTable(PhoenixTransactionContext phoenixTransactionContext, HTableInterface htable, PTable pTable) {
-        return new TephraTransactionTable(phoenixTransactionContext, htable, pTable);
+    public static long convertToMilliseconds(long serverTimeStamp) {
+        return serverTimeStamp / MAX_TRANSACTIONS_PER_MILLISECOND;
     }
     
     // we resolve transactional tables at the txn read pointer
@@ -77,16 +98,58 @@ 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;
 	}
+	
+    // Convert HBase Delete into Put so that it can be undone if transaction is rolled back
+    public static Mutation convertIfDelete(Mutation mutation) throws IOException {
+        if (mutation instanceof Delete) {
+            Put deleteMarker = null;
+            for (Map.Entry<byte[],List<Cell>> entry : mutation.getFamilyCellMap().entrySet()) {
+                byte[] family = entry.getKey();
+                List<Cell> familyCells = entry.getValue();
+                if (familyCells.size() == 1) {
+                    if (CellUtil.isDeleteFamily(familyCells.get(0))) {
+                        if (deleteMarker == null) {
+                            deleteMarker = new Put(mutation.getRow());
+                        }
+                        deleteMarker.add(newDeleteFamilyMarker(
+                                deleteMarker.getRow(), 
+                                family, 
+                                familyCells.get(0).getTimestamp()));
+                    }
+                } else {
+                    for (Cell cell : familyCells) {
+                        if (CellUtil.isDeleteColumns(cell)) {
+                            if (deleteMarker == null) {
+                                deleteMarker = new Put(mutation.getRow());
+                            }
+                            deleteMarker.add(newDeleteColumnMarker(
+                                    deleteMarker.getRow(),
+                                    family,
+                                    CellUtil.cloneQualifier(cell), 
+                                    cell.getTimestamp()));
+                        }
+                    }
+                }
+            }
+            if (deleteMarker != null) {
+                for (Map.Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) {
+                    deleteMarker.setAttribute(entry.getKey(), entry.getValue());
+                }
+                mutation = deleteMarker;
+            }
+        }
+        return mutation;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 76757b0..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
@@ -56,7 +56,6 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 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;
@@ -64,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;
@@ -262,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/8eaca121/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/8eaca121/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 580becb..0ea63e7 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/8eaca121/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/8eaca121/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 1ec07b6..a06fd69 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
@@ -780,7 +780,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 (HTableInterface htable = mutationState.getHTable(table)) {
             byte[] markerRowKey = Bytes.toBytes("TO_DELETE");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8eaca121/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 {


[03/21] phoenix git commit: PHOENIX-2715 Query Log (Ankit Singhal)

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/log/TableLogWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/log/TableLogWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/log/TableLogWriter.java
new file mode 100644
index 0000000..c102855
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/TableLogWriter.java
@@ -0,0 +1,125 @@
+/*
+ * 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.log;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Map.Entry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.Determinism;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Writes RingBuffer log event into table 
+ * 
+ */
+public class TableLogWriter implements LogWriter {
+    private static final Log LOG = LogFactory.getLog(LogWriter.class);
+    private Connection connection;
+    private boolean isClosed;
+    private Table table;
+    private Configuration config;
+
+    public TableLogWriter(Configuration configuration) {
+        this.config = configuration;
+        try {
+            this.connection = ConnectionFactory.createConnection(configuration);
+            table = this.connection.getTable(SchemaUtil.getPhysicalTableName(
+                    SchemaUtil.getTableNameAsBytes(SYSTEM_CATALOG_SCHEMA, SYSTEM_LOG_TABLE), config));
+        } catch (Exception e) {
+            LOG.warn("Unable to initiate LogWriter for writing query logs to table");
+        }
+    }
+
+    @Override
+    public void write(RingBufferEvent event) throws SQLException, IOException {
+        if(isClosed()){
+            LOG.warn("Unable to commit query log as Log committer is already closed");
+            return;
+        }
+        if (table == null || connection == null) {
+            LOG.warn("Unable to commit query log as connection was not initiated ");
+            return;
+        }
+        ImmutableMap<QueryLogInfo, Object> queryInfo=event.getQueryInfo();
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        Put put =new Put(Bytes.toBytes(event.getQueryId()));
+        for (Entry<QueryLogInfo, Object> entry : queryInfo.entrySet()) {
+            if (entry.getKey().logLevel.ordinal() <= event.getConnectionLogLevel().ordinal()) {
+                LiteralExpression expression = LiteralExpression.newConstant(entry.getValue(), entry.getKey().dataType,
+                        Determinism.ALWAYS);
+                expression.evaluate(null, ptr);
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes(entry.getKey().columnName),
+                        ByteUtil.copyKeyBytesIfNecessary(ptr));
+            }
+        }
+        
+        if (QueryLogInfo.QUERY_STATUS_I.logLevel.ordinal() <= event.getConnectionLogLevel().ordinal()
+                && (event.getLogState() == QueryLogState.COMPLETED || event.getLogState() == QueryLogState.FAILED)) {
+            LiteralExpression expression = LiteralExpression.newConstant(event.getLogState().toString(),
+                    QueryLogInfo.QUERY_STATUS_I.dataType, Determinism.ALWAYS);
+            expression.evaluate(null, ptr);
+            put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                    Bytes.toBytes(QueryLogInfo.QUERY_STATUS_I.columnName), ByteUtil.copyKeyBytesIfNecessary(ptr));
+        }
+        put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+        table.put(put);
+        
+    }
+    
+    @Override
+    public void close() throws IOException {
+        if(isClosed()){
+            return;
+        }
+        isClosed=true;
+        try {
+            if (table != null) {
+                table.close();
+            }
+            if (connection != null && !connection.isClosed()) {
+                //It should internally close all the statements
+                connection.close();
+            }
+        } catch (IOException e) {
+            // TODO Ignore?
+        }
+    }
+    
+    public boolean isClosed(){
+        return isClosed;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ReadMetricQueue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ReadMetricQueue.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ReadMetricQueue.java
index 4fd1194..c008635 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ReadMetricQueue.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ReadMetricQueue.java
@@ -1,27 +1,21 @@
 /*
- * 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.
+ * 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.monitoring;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
@@ -44,6 +38,8 @@ public class ReadMetricQueue {
 
     private final ConcurrentMap<MetricKey, Queue<CombinableMetric>> metricsMap = new ConcurrentHashMap<>();
 
+    private final List<ScanMetricsHolder> scanMetricsHolderList = new ArrayList<ScanMetricsHolder>();
+
     private final boolean isRequestMetricsEnabled;
 
     public ReadMetricQueue(boolean isRequestMetricsEnabled) {
@@ -85,7 +81,7 @@ public class ReadMetricQueue {
         }
         return publishedMetrics;
     }
-    
+
     public void clearMetrics() {
         metricsMap.clear(); // help gc
     }
@@ -177,8 +173,18 @@ public class ReadMetricQueue {
         return q;
     }
 
-	public boolean isRequestMetricsEnabled() {
-		return isRequestMetricsEnabled;
-	}
+    public boolean isRequestMetricsEnabled() {
+        return isRequestMetricsEnabled;
+    }
+    
+    public void addScanHolder(ScanMetricsHolder holder){
+        scanMetricsHolderList.add(holder);
+    }
+
+    public List<ScanMetricsHolder> getScanMetricsHolderList() {
+        return scanMetricsHolderList;
+    }
+    
+    
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ScanMetricsHolder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ScanMetricsHolder.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ScanMetricsHolder.java
index 6bcd402..9125cd8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ScanMetricsHolder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/ScanMetricsHolder.java
@@ -17,20 +17,23 @@
  */
 package org.apache.phoenix.monitoring;
 
-import static org.apache.phoenix.monitoring.MetricType.COUNT_RPC_CALLS;
-import static org.apache.phoenix.monitoring.MetricType.COUNT_REMOTE_RPC_CALLS;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_BYTES_IN_REMOTE_RESULTS;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_BYTES_REGION_SERVER_RESULTS;
 import static org.apache.phoenix.monitoring.MetricType.COUNT_MILLS_BETWEEN_NEXTS;
 import static org.apache.phoenix.monitoring.MetricType.COUNT_NOT_SERVING_REGION_EXCEPTION;
-import static org.apache.phoenix.monitoring.MetricType.COUNT_BYTES_REGION_SERVER_RESULTS;
-import static org.apache.phoenix.monitoring.MetricType.COUNT_BYTES_IN_REMOTE_RESULTS;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_REMOTE_RPC_CALLS;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_REMOTE_RPC_RETRIES;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_ROWS_FILTERED;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_ROWS_SCANNED;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_RPC_CALLS;
+import static org.apache.phoenix.monitoring.MetricType.COUNT_RPC_RETRIES;
 import static org.apache.phoenix.monitoring.MetricType.COUNT_SCANNED_REGIONS;
 
-import org.apache.hadoop.hbase.client.Scan;
+import java.io.IOException;
+import java.util.Map;
 
-import static org.apache.phoenix.monitoring.MetricType.COUNT_RPC_RETRIES;
-import static org.apache.phoenix.monitoring.MetricType.COUNT_REMOTE_RPC_RETRIES;
-import static org.apache.phoenix.monitoring.MetricType.COUNT_ROWS_SCANNED;
-import static org.apache.phoenix.monitoring.MetricType.COUNT_ROWS_FILTERED;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.JsonMapper;
 
 public class ScanMetricsHolder {
 
@@ -45,9 +48,11 @@ public class ScanMetricsHolder {
     private final CombinableMetric countOfRemoteRPCRetries;
     private final CombinableMetric countOfRowsScanned;
     private final CombinableMetric countOfRowsFiltered;
+    private  Map<String, Long> scanMetricMap;
+    private Object scan;
 
     private static final ScanMetricsHolder NO_OP_INSTANCE =
-            new ScanMetricsHolder(new ReadMetricQueue(false), "");
+            new ScanMetricsHolder(new ReadMetricQueue(false), "",null);
 
     public static ScanMetricsHolder getInstance(ReadMetricQueue readMetrics, String tableName,
             Scan scan, boolean isRequestMetricsEnabled) {
@@ -55,10 +60,12 @@ public class ScanMetricsHolder {
             return NO_OP_INSTANCE;
         }
         scan.setScanMetricsEnabled(true);
-        return new ScanMetricsHolder(readMetrics, tableName);
+        return new ScanMetricsHolder(readMetrics, tableName, scan);
     }
 
-    private ScanMetricsHolder(ReadMetricQueue readMetrics, String tableName) {
+    private ScanMetricsHolder(ReadMetricQueue readMetrics, String tableName,Scan scan) {
+        readMetrics.addScanHolder(this);
+        this.scan=scan;
         countOfRPCcalls = readMetrics.allotMetric(COUNT_RPC_CALLS, tableName);
         countOfRemoteRPCcalls = readMetrics.allotMetric(COUNT_REMOTE_RPC_CALLS, tableName);
         sumOfMillisSecBetweenNexts = readMetrics.allotMetric(COUNT_MILLS_BETWEEN_NEXTS, tableName);
@@ -118,4 +125,21 @@ public class ScanMetricsHolder {
         return countOfRowsScanned;
     }
 
+    public Map<String, Long> getScanMetricMap() {
+        return scanMetricMap;
+    }
+
+    public void setScanMetricMap(Map<String, Long> scanMetricMap) {
+        this.scanMetricMap = scanMetricMap;
+    }
+    
+    @Override
+    public String toString() {
+        try {
+            return "{\"scan\":" + scan + ", \"scanMetrics\":" + JsonMapper.writeObjectAsString(scanMetricMap) + "}";
+        } catch (IOException e) {
+            return "{\"Exception while converting scan metrics to Json\":\"" + e.getMessage() + "\"}";
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 90f8089..0b72ada 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
@@ -30,12 +30,14 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.MutationPlan;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.log.QueryLoggerDisruptor;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
@@ -146,4 +148,8 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     void upgradeSystemTables(String url, Properties props) throws SQLException;
     
     public Configuration getConfiguration();
+
+    public User getUser();
+
+    public QueryLoggerDisruptor getQueryDisruptor();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 6df2f80..8c7441a 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
@@ -124,6 +124,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -183,6 +184,7 @@ import org.apache.phoenix.iterate.TableResultIterator.RenewLeaseStatus;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.log.QueryLoggerDisruptor;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.protobuf.ProtobufUtil;
@@ -267,6 +269,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // don't need.
     private final ReadOnlyProps props;
     private final String userName;
+    private final User user;
     private final ConcurrentHashMap<ImmutableBytesWritable,ConnectionQueryServices> childServices;
     private final GuidePostsCache tableStatsCache;
 
@@ -336,6 +339,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
+    private QueryLoggerDisruptor queryDisruptor;
 
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
@@ -372,6 +376,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         ConfigUtil.setReplicationConfigIfAbsent(this.config);
         this.props = new ReadOnlyProps(this.config.iterator());
         this.userName = connectionInfo.getPrincipal();
+        this.user = connectionInfo.getUser();
         this.latestMetaData = newEmptyMetaData();
         // TODO: should we track connection wide memory usage or just org-wide usage?
         // If connection-wide, create a MemoryManager here, otherwise just use the one from the delegate
@@ -396,6 +401,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         this.maxConnectionsAllowed = config.getInt(QueryServices.CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS,
             QueryServicesOptions.DEFAULT_CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS);
         this.shouldThrottleNumConnections = (maxConnectionsAllowed > 0);
+        try {
+            this.queryDisruptor = new QueryLoggerDisruptor(this.config);
+        } catch (SQLException e) {
+            logger.warn("Unable to initiate qeuery logging service !!");
+            e.printStackTrace();
+        }
 
     }
 
@@ -477,6 +488,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
             closed = true;
             GLOBAL_QUERY_SERVICES_COUNTER.decrement();
+            try {
+                if (this.queryDisruptor != null) {
+                    this.queryDisruptor.close();
+                }
+            } catch (Exception e) {
+                // Ignore
+            }
             SQLException sqlE = null;
             try {
                 // Attempt to return any unused sequences.
@@ -2611,7 +2629,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.createStatement().execute(QueryConstants.CREATE_FUNCTION_METADATA);
         } catch (TableAlreadyExistsException ignore) {}
-
+        try {
+            metaConnection.createStatement().execute(QueryConstants.CREATE_LOG_METADATA);
+        } catch (TableAlreadyExistsException ignore) {}
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
             createSysMutexTableIfNotExists(hbaseAdmin, ConnectionQueryServicesImpl.this.getProps());
@@ -2966,6 +2986,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
+            try {
+                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
+            } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
             ConnectionQueryServicesImpl.this.upgradeRequired.set(false);
             success = true;
         } catch (UpgradeInProgressException | UpgradeNotRequiredException e) {
@@ -4063,6 +4086,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public String getUserName() {
         return userName;
     }
+    
+    @Override
+    public User getUser() {
+        return user;
+    }
 
     private void checkClosed() {
         if (closed) {
@@ -4488,4 +4516,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public Configuration getConfiguration() {
         return config;
     }
+
+    @Override
+    public QueryLoggerDisruptor getQueryDisruptor() {
+        return this.queryDisruptor;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 c510b5a..ad354d1 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -55,6 +56,7 @@ import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.log.QueryLoggerDisruptor;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.schema.FunctionNotFoundException;
@@ -112,10 +114,13 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     private final Map<String, List<HRegionLocation>> tableSplits = Maps.newHashMap();
     private final GuidePostsCache guidePostsCache;
     private final Configuration config;
+
+    private User user;
     
     public ConnectionlessQueryServicesImpl(QueryServices services, ConnectionInfo connInfo, Properties info) {
         super(services);
         userName = connInfo.getPrincipal();
+        user = connInfo.getUser();
         metaData = newEmptyMetaData();
 
         // Use KeyValueBuilder that builds real KeyValues, as our test utils require this
@@ -328,6 +333,9 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
                 } catch (NewerTableAlreadyExistsException ignore) {
                 }
+                try {
+                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
+                } catch (NewerTableAlreadyExistsException ignore) {}
             } catch (SQLException e) {
                 sqlE = e;
             } finally {
@@ -664,4 +672,14 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     public Configuration getConfiguration() {
         return config;
     }
+
+    @Override
+    public User getUser() {
+        return user;
+    }
+
+    @Override
+    public QueryLoggerDisruptor getQueryDisruptor() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 05d1af6..f5c8a59 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
@@ -30,12 +30,14 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.MutationPlan;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.log.QueryLoggerDisruptor;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.schema.PColumn;
@@ -351,4 +353,16 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public Configuration getConfiguration() {
         return getDelegate().getConfiguration();
     }
+
+    @Override
+    public User getUser() {
+        return getDelegate().getUser();
+    }
+
+    @Override
+    public QueryLoggerDisruptor getQueryDisruptor() {
+        return getDelegate().getQueryDisruptor();
+    }
+    
+    
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 7607388..ae12e01 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
@@ -109,6 +109,20 @@ 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;
 
@@ -124,6 +138,7 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TableProperty;
 
 
 /**
@@ -395,10 +410,40 @@ public interface QueryConstants {
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
+    
+    public static final String CREATE_LOG_METADATA =
+            "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_LOG_TABLE + "\"(\n" +
+             // Pk columns
+            TENANT_ID + " VARCHAR ," +
+            QUERY_ID + " VARCHAR NOT NULL,\n" +
+            USER + " VARCHAR , \n" +
+            CLIENT_IP + " VARCHAR, \n" +
+            // Function metadata (will be null for argument row)
+            QUERY +  " VARCHAR, \n" +
+            EXPLAIN_PLAN + " VARCHAR, \n" +
+            // Argument metadata (will be null for function row)
+            START_TIME + " TIMESTAMP, \n" +
+            TOTAL_EXECUTION_TIME + " BIGINT, \n" +
+            NO_OF_RESULTS_ITERATED + " BIGINT, \n" +
+            QUERY_STATUS + " VARCHAR, \n" +
+            EXCEPTION_TRACE + " VARCHAR, \n" +
+            GLOBAL_SCAN_DETAILS + " VARCHAR, \n" +
+            BIND_PARAMETERS + " VARCHAR, \n" +
+            SCAN_METRICS_JSON + " VARCHAR, \n" +
+            " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (QUERY_ID))\n" +
+            HConstants.VERSIONS + "= " + MetaDataProtocol.DEFAULT_LOG_VERSIONS + ",\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n"+
+            // Install split policy to prevent a tenant's metadata from being split across regions.
+            HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
+            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE+ ",\n" + 
+            HColumnDescriptor.TTL + "=" + MetaDataProtocol.DEFAULT_LOG_TTL+",\n"+
+            TableProperty.COLUMN_ENCODED_BYTES.toString()+" = 0";
+    
     public static final byte[] OFFSET_FAMILY = "f_offset".getBytes();
     public static final byte[] OFFSET_COLUMN = "c_offset".getBytes();
     public static final String LAST_SCAN = "LAST_SCAN";
     public static final byte[] UPGRADE_MUTEX = "UPGRADE_MUTEX".getBytes();
     public static final String HASH_JOIN_CACHE_RETRIES = "hashjoin.client.retries.number";
     public static final int DEFAULT_HASH_JOIN_CACHE_RETRIES = 5;
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 0b18aaa..43b9e5a 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
@@ -301,6 +301,10 @@ public interface QueryServices extends SQLCloseable {
     // Whether to enable cost-based-decision in the query optimizer
     public static final String COST_BASED_OPTIMIZER_ENABLED = "phoenix.costbased.optimizer.enabled";
     public static final String SMALL_SCAN_THRESHOLD_ATTRIB = "phoenix.query.smallScanThreshold";
+    public static final String LOG_LEVEL = "phoenix.log.level";
+    public static final String LOG_BUFFER_SIZE = "phoenix.log.buffer.size";
+    public static final String LOG_BUFFER_WAIT_STRATEGY = "phoenix.log.wait.strategy";
+    public static final String LOG_SAMPLE_RATE = "phoenix.log.sample.rate";
 
     /**
      * Get executor service used for parallel scans

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/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 961ab9f..58c9812 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
@@ -47,6 +47,8 @@ import static org.apache.phoenix.query.QueryServices.IS_NAMESPACE_MAPPING_ENABLE
 import static org.apache.phoenix.query.QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE;
 import static org.apache.phoenix.query.QueryServices.KEEP_ALIVE_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.LOG_LEVEL;
+import static org.apache.phoenix.query.QueryServices.LOG_SAMPLE_RATE;
 import static org.apache.phoenix.query.QueryServices.MASTER_INFO_PORT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_PERC_ATTRIB;
@@ -107,6 +109,7 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ClientRpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
+import org.apache.phoenix.log.LogLevel;
 import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableRefFactory;
@@ -347,6 +350,8 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_ENABLE_SERVER_UPSERT_SELECT = false;
 
     public static final boolean DEFAULT_COST_BASED_OPTIMIZER_ENABLED = false;
+    public static final String DEFAULT_LOGGING_LEVEL = LogLevel.OFF.name();
+    public static final String DEFAULT_LOG_SAMPLE_RATE = "1.0";
 
     private final Configuration config;
 
@@ -428,7 +433,9 @@ public class QueryServicesOptions {
             .setIfUnset(USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
             .setIfUnset(COST_BASED_OPTIMIZER_ENABLED, DEFAULT_COST_BASED_OPTIMIZER_ENABLED)
             .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING)
-            .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED);
+            .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED)
+            .setIfUnset(LOG_LEVEL,  DEFAULT_LOGGING_LEVEL)
+            .setIfUnset(LOG_SAMPLE_RATE,  DEFAULT_LOG_SAMPLE_RATE);
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
         // it to 1, so we'll change it.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b291068b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 73964c0..970428a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -120,6 +120,7 @@
     <jackson.version>${cdh.jackson.version}</jackson.version>
     <antlr.version>3.5.2</antlr.version>
     <log4j.version>1.2.17</log4j.version>
+    <disruptor.version>3.3.6</disruptor.version>
     <slf4j.version>${cdh.slf4j.version}</slf4j.version>
     <protobuf-java.version>2.5.0</protobuf-java.version>
     <commons-io.version>2.1</commons-io.version>
@@ -994,6 +995,11 @@
         <artifactId>javax.servlet-api</artifactId>
         <version>${servlet.api.version}</version>
       </dependency>
+       <dependency>
+        <groupId>com.lmax</groupId>
+        <artifactId>disruptor</artifactId>
+        <version>${disruptor.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 


[11/21] phoenix git commit: PHOENIX-4366 Rebuilding a local index fails sometimes (addendum)

Posted by pb...@apache.org.
PHOENIX-4366 Rebuilding a local index fails sometimes (addendum)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 0535a17ed64e681b801f2b48648183b224ff8b9b
Parents: a874df3
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Apr 11 23:20:08 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:27:44 2018 +0100

----------------------------------------------------------------------
 .../NonAggregateRegionScannerFactory.java       | 36 +++++++++++---------
 1 file changed, 20 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0535a17e/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 90ea025..ba6a08f 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
@@ -18,8 +18,15 @@
 
 package org.apache.phoenix.iterate;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getMinMaxQualifiersFromScan;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
@@ -61,15 +68,8 @@ import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.phoenix.util.EncodedColumnsUtil.getMinMaxQualifiersFromScan;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
 
@@ -102,13 +102,17 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
     boolean useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
 
     Set<KeyValueColumnExpression> arrayKVRefs = Sets.newHashSet();
+    KeyValueSchema kvSchema = null;
+    ValueBitSet kvSchemaBitSet = null;
     Expression[] arrayFuncRefs = deserializeArrayPositionalExpressionInfoFromScan(scan, innerScanner, arrayKVRefs);
-    KeyValueSchema.KeyValueSchemaBuilder builder = new KeyValueSchema.KeyValueSchemaBuilder(0);
-    for (Expression expression : arrayFuncRefs) {
-        builder.addField(expression);
+    if (arrayFuncRefs != null) {
+        KeyValueSchema.KeyValueSchemaBuilder builder = new KeyValueSchema.KeyValueSchemaBuilder(0);
+        for (Expression expression : arrayFuncRefs) {
+            builder.addField(expression);
+        }
+        kvSchema = builder.build();
+        kvSchemaBitSet = ValueBitSet.newInstance(kvSchema);
     }
-    KeyValueSchema kvSchema = builder.build();
-    ValueBitSet kvSchemaBitSet = ValueBitSet.newInstance(kvSchema);
     TupleProjector tupleProjector = null;
     Region dataRegion = null;
     IndexMaintainer indexMaintainer = null;


[17/21] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

Posted by pb...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 5cb14d6..fa5d7e2 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
@@ -170,6 +170,7 @@ import org.apache.phoenix.exception.RetriableUpgradeException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.exception.UpgradeInProgressException;
+import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.exception.UpgradeNotRequiredException;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -1041,23 +1042,69 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     /**
      *
-     * @param tableName
+     * @param physicalTableName
+     * @param tableType
+     * @param props
+     * @param families
      * @param splits
-     * @param modifyExistingMetaData TODO
+     * @param modifyExistingMetaData
+     * @param isNamespaceMapped
+     * @param isDoNotUpgradePropSet
      * @return true if table was created and false if it already exists
      * @throws SQLException
      */
     private HTableDescriptor ensureTableCreated(byte[] physicalTableName, PTableType tableType, Map<String, Object> props,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean modifyExistingMetaData,
-            boolean isNamespaceMapped) throws SQLException {
+            boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws SQLException {
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
-            final String znode = this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+            final String znode = this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
             logger.debug("Found quorum: " + quorum + ":" + znode);
+
+            if (isMetaTable) {
+                if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, this.getProps())) {
+                    try {
+                        // SYSTEM namespace needs to be created via HBase APIs because "CREATE SCHEMA" statement tries to write
+                        // its metadata in SYSTEM:CATALOG table. Without SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+                        ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+                    } catch (PhoenixIOException e) {
+                        // We could either:
+                        // 1) Not access the NS descriptor. The NS may or may not exist at this point
+                        // 2) We could not create the NS
+                        // Regardless of the case 1 or 2, if we eventually try to migrate SYSTEM tables to the SYSTEM
+                        // namespace using the {@link ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+                        // if the NS does not exist, we will error as expected, or
+                        // if the NS does exist and tables are already mapped, the check will exit gracefully
+                    }
+                    if (admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, false))) {
+                        // SYSTEM.CATALOG exists, so at this point, we have 3 cases:
+                        // 1) If server-side namespace mapping is disabled, throw Inconsistent namespace mapping exception
+                        // 2) If server-side namespace mapping is enabled and SYSCAT needs to be upgraded, upgrade SYSCAT
+                        //    and also migrate SYSTEM tables to the SYSTEM namespace
+                        // 3. If server-side namespace mapping is enabled and SYSCAT doesn't need to be upgraded, we still
+                        //    need to migrate SYSTEM tables to the SYSTEM namespace using the
+                        //    {@link ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
+                        //    {@link upgradeSystemTables(String, Properties)})
+                        checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+                        // Thrown so we can force an upgrade which will just migrate SYSTEM tables to the SYSTEM namespace
+                        throw new UpgradeRequiredException(MIN_SYSTEM_TABLE_TIMESTAMP);
+                    }
+                } else if (admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true))) {
+                    // If SYSTEM:CATALOG exists, but client-side namespace mapping for SYSTEM tables is disabled, throw an exception
+                    throw new SQLExceptionInfo.Builder(
+                      SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES)
+                      .setMessage("Cannot initiate connection as "
+                        + SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true)
+                        + " is found but client does not have "
+                        + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                      .build().buildException();
+                }
+            }
+
             try {
                 existingDesc = admin.getTableDescriptor(physicalTableName);
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -1075,6 +1122,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     splits, isNamespaceMapped);
 
             if (!tableExist) {
+                if (isMetaTable && !isUpgradeRequired() && (!isAutoUpgradeEnabled || isDoNotUpgradePropSet)) {
+                    // Disallow creating the SYSTEM.CATALOG or SYSTEM:CATALOG HBase table
+                    throw new UpgradeRequiredException();
+                }
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
                         PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
@@ -1092,9 +1143,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } catch (TableExistsException e) {
                     // We can ignore this, as it just means that another client beat us
                     // to creating the HBase metadata.
+                    if (isMetaTable && !isUpgradeRequired()) {
+                        checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
+                    }
                     return null;
                 }
-                if (isMetaTable) {
+                if (isMetaTable && !isUpgradeRequired()) {
                     checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
                     /*
                      * Now we modify the table to add the split policy, since we know that the client and
@@ -1106,7 +1160,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
                 return null;
             } else {
-                if (isMetaTable) {
+                if (isMetaTable && !isUpgradeRequired()) {
                     checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
@@ -1120,7 +1174,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
 
-
                 if (!modifyExistingMetaData) {
                     return existingDesc; // Caller already knows that no metadata was changed
                 }
@@ -1143,7 +1196,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return null; // Indicate that no metadata was changed
                 }
 
-                modifyTable(physicalTableName, newDesc, true);
+                // Do not call modifyTable for SYSTEM tables
+                if (tableType != PTableType.SYSTEM) {
+                    modifyTable(physicalTableName, newDesc, true);
+                }
                 return newDesc;
             }
 
@@ -1198,6 +1254,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         boolean isIncompatible = false;
         int minHBaseVersion = Integer.MAX_VALUE;
         boolean isTableNamespaceMappingEnabled = false;
+        long systemCatalogTimestamp = Long.MAX_VALUE;
         HTableInterface ht = null;
         try {
             List<HRegionLocation> locations = this
@@ -1214,36 +1271,44 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
 
             ht = this.getTable(metaTable);
-            final Map<byte[], Long> results =
-                    ht.coprocessorService(MetaDataService.class, null, null, new Batch.Call<MetaDataService,Long>() {
+            final Map<byte[], GetVersionResponse> results =
+                    ht.coprocessorService(MetaDataService.class, null, null, new Batch.Call<MetaDataService,GetVersionResponse>() {
                         @Override
-                        public Long call(MetaDataService instance) throws IOException {
+                        public GetVersionResponse call(MetaDataService instance) throws IOException {
                             ServerRpcController controller = new ServerRpcController();
                             BlockingRpcCallback<GetVersionResponse> rpcCallback =
-                                    new BlockingRpcCallback<GetVersionResponse>();
+                                    new BlockingRpcCallback<>();
                             GetVersionRequest.Builder builder = GetVersionRequest.newBuilder();
                             builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                             instance.getVersion(controller, builder.build(), rpcCallback);
                             if(controller.getFailedOn() != null) {
                                 throw controller.getFailedOn();
                             }
-                            return rpcCallback.get().getVersion();
+                            return rpcCallback.get();
                         }
                     });
-            for (Map.Entry<byte[],Long> result : results.entrySet()) {
+            for (Map.Entry<byte[],GetVersionResponse> result : results.entrySet()) {
                 // This is the "phoenix.jar" is in-place, but server is out-of-sync with client case.
-                long version = result.getValue();
-                isTableNamespaceMappingEnabled |= MetaDataUtil.decodeTableNamespaceMappingEnabled(version);
+                GetVersionResponse versionResponse = result.getValue();
+                long serverJarVersion = versionResponse.getVersion();
+                isTableNamespaceMappingEnabled |= MetaDataUtil.decodeTableNamespaceMappingEnabled(serverJarVersion);
 
-                if (!isCompatible(result.getValue())) {
+                if (!isCompatible(serverJarVersion)) {
                     isIncompatible = true;
                     HRegionLocation name = regionMap.get(result.getKey());
                     buf.append(name);
                     buf.append(';');
                 }
-                hasIndexWALCodec &= hasIndexWALCodec(result.getValue());
-                if (minHBaseVersion > MetaDataUtil.decodeHBaseVersion(result.getValue())) {
-                    minHBaseVersion = MetaDataUtil.decodeHBaseVersion(result.getValue());
+                hasIndexWALCodec &= hasIndexWALCodec(serverJarVersion);
+                if (minHBaseVersion > MetaDataUtil.decodeHBaseVersion(serverJarVersion)) {
+                    minHBaseVersion = MetaDataUtil.decodeHBaseVersion(serverJarVersion);
+                }
+                // In case this is the first time connecting to this cluster, the system catalog table does not have an
+                // entry for itself yet, so we cannot get the timestamp and this will not be returned from the
+                // GetVersionResponse message object
+                if (versionResponse.hasSystemCatalogTimestamp()) {
+                    systemCatalogTimestamp = systemCatalogTimestamp < versionResponse.getSystemCatalogTimestamp() ?
+                      systemCatalogTimestamp: versionResponse.getSystemCatalogTimestamp();
                 }
             }
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
@@ -1274,6 +1339,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             buf.setLength(buf.length()-1);
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.OUTDATED_JARS).setMessage(buf.toString()).build().buildException();
         }
+        if (systemCatalogTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
+            throw new UpgradeRequiredException(systemCatalogTimestamp);
+        }
     }
 
     /**
@@ -1334,14 +1402,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
         HTableDescriptor desc = ensureTableCreated(physicalIndexName, PTableType.TABLE, tableProps, families, splits,
-                false, isNamespaceMapped);
+                false, isNamespaceMapped, false);
         if (desc != null) {
             if (!Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_BYTES)))) {
                 String fullTableName = Bytes.toString(physicalIndexName);
                 throw new TableAlreadyExistsException(
-                        "Unable to create shared physical table for indexes on views.",
-                        SchemaUtil.getSchemaNameFromFullName(fullTableName),
-                        SchemaUtil.getTableNameFromFullName(fullTableName));
+                  SchemaUtil.getSchemaNameFromFullName(fullTableName),
+                  SchemaUtil.getTableNameFromFullName(fullTableName),
+                  "Unable to create shared physical table for indexes on views.");
             }
         }
     }
@@ -1409,8 +1477,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     @Override
     public MetaDataMutationResult createTable(final List<Mutation> tableMetaData, final byte[] physicalTableName,
             PTableType tableType, Map<String, Object> tableProps,
-            final List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean isNamespaceMapped, final boolean allocateIndexId)
-                    throws SQLException {
+            final List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean isNamespaceMapped,
+      final boolean allocateIndexId, final boolean isDoNotUpgradePropSet) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         Mutation m = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetaData);
         byte[] key = m.getRow();
@@ -1430,7 +1498,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
             // For tables and indexes, this will create the metadata if it doesn't already exist
-            ensureTableCreated(tableName, tableType, tableProps, families, splits, true, isNamespaceMapped);
+            ensureTableCreated(tableName, tableType, tableProps, families, splits, true, isNamespaceMapped, isDoNotUpgradePropSet);
         }
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         if (tableType == PTableType.INDEX) { // Index on view
@@ -2436,30 +2504,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             openConnection();
                             hConnectionEstablished = true;
                             boolean isDoNotUpgradePropSet = UpgradeUtil.isNoUpgradeSet(props);
-                            try (HBaseAdmin admin = getAdmin()) {
-                                boolean mappedSystemCatalogExists = admin
-                                        .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
-                                if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                        ConnectionQueryServicesImpl.this.getProps())) {
-                                    if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
-                                        checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
-                                    }
-
-                                    // If SYSTEM tables exist, they are migrated to HBase SYSTEM namespace
-                                    // If they don't exist, this method will create HBase SYSTEM namespace and return
-                                    ensureSystemTablesMigratedToSystemNamespace(ConnectionQueryServicesImpl.this.getProps());
-                                } else if (mappedSystemCatalogExists) {
-                                    throw new SQLExceptionInfo.Builder(
-                                            SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES)
-                                            .setMessage("Cannot initiate connection as "
-                                                    + SchemaUtil.getPhysicalTableName(
-                                                    SYSTEM_CATALOG_NAME_BYTES, true)
-                                                    + " is found but client does not have "
-                                                    + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException();
-                                }
-                            }
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
                                     Long.toString(getSystemTableVersion()));
@@ -2508,38 +2552,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                         initializationException = e;
                                     }
                                     return null;
+                                } catch (UpgradeRequiredException e) {
+                                    // This will occur in 3 cases:
+                                    // 1. SYSTEM.CATALOG does not exist and we don't want to allow the user to create it i.e.
+                                    //    !isAutoUpgradeEnabled or isDoNotUpgradePropSet is set
+                                    // 2. SYSTEM.CATALOG exists and its timestamp < MIN_SYSTEM_TABLE_TIMESTAMP
+                                    // 3. SYSTEM.CATALOG exists, but client and server-side namespace mapping is enabled so
+                                    //    we need to migrate SYSTEM tables to the SYSTEM namespace
+                                    setUpgradeRequired();
                                 }
 
-                                // HBase Namespace SYSTEM is created by {@link ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method
-                                // This statement will create its entry in SYSCAT table, so that GRANT/REVOKE commands can work
-                                // with SYSTEM Namespace. (See PHOENIX-4227 https://issues.apache.org/jira/browse/PHOENIX-4227)
-                                if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                        ConnectionQueryServicesImpl.this.getProps())) {
-                                    try {
-                                        metaConnection.createStatement().execute("CREATE SCHEMA IF NOT EXISTS "
-                                                + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
-                                    } catch (NewerSchemaAlreadyExistsException e) {
-                                        // Older clients with appropriate perms may try getting a new connection
-                                        // This results in NewerSchemaAlreadyExistsException, so we can safely ignore it here
-                                    } catch (PhoenixIOException e) {
-                                        if (!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), AccessDeniedException.class))) {
-                                            // Ignore ADE
-                                        } else {
-                                            throw e;
-                                        }
-                                    }
-                                }
                                 if (!ConnectionQueryServicesImpl.this.upgradeRequired.get()) {
                                     createOtherSystemTables(metaConnection, hBaseAdmin);
+                                    // In case namespace mapping is enabled and system table to system namespace mapping is also enabled,
+                                    // create an entry for the SYSTEM namespace in the SYSCAT table, so that GRANT/REVOKE commands can work
+                                    // with SYSTEM Namespace
+                                    createSchemaIfNotExistsSystemNSMappingEnabled(metaConnection);
                                 } else if (isAutoUpgradeEnabled && !isDoNotUpgradePropSet) {
+                                    // Upgrade is required and we are allowed to automatically upgrade
                                     upgradeSystemTables(url, props);
+                                } else {
+                                    // We expect the user to manually run the "EXECUTE UPGRADE" command first.
+                                    // This exception will get caught below as a RetriableUpgradeException
+                                    throw new UpgradeRequiredException();
                                 }
                             }
                             scheduleRenewLeaseTasks();
                             success = true;
                         } catch (RetriableUpgradeException e) {
-                            // Don't set it as initializationException because otherwise the clien't won't be able
-                            // to retry establishing connection.
+                            // Set success to true and don't set the exception as an initializationException,
+                            // because otherwise the client won't be able to retry establishing the connection.
+                            success = true;
                             throw e;
                         } catch (Exception e) {
                             if (e instanceof SQLException) {
@@ -2580,7 +2623,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
-    void createSysMutexTableIfNotExists(HBaseAdmin admin, ReadOnlyProps props) throws IOException, SQLException {
+    void createSysMutexTableIfNotExists(HBaseAdmin admin) throws IOException, SQLException {
         try {
             if(admin.tableExists(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME) || admin.tableExists(TableName.valueOf(
                     PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME))) {
@@ -2588,7 +2631,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 return;
             }
             final TableName mutexTableName = SchemaUtil.getPhysicalTableName(
-                PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
+                PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, this.getProps());
             HTableDescriptor tableDesc = new HTableDescriptor(mutexTableName);
             HColumnDescriptor columnDesc = new HColumnDescriptor(
                     PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES);
@@ -2640,7 +2683,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (TableAlreadyExistsException ignore) {}
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
-            createSysMutexTableIfNotExists(hbaseAdmin, ConnectionQueryServicesImpl.this.getProps());
+            createSysMutexTableIfNotExists(hbaseAdmin);
         } catch (IOException exception) {
             logger.error("Failed to created SYSMUTEX table. Upgrade or migration is not possible without it. Please retry.");
             throw exception;
@@ -2648,6 +2691,265 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     /**
+     * Create an entry for the SYSTEM namespace in the SYSCAT table in case namespace mapping is enabled and system table
+     * to system namespace mapping is also enabled. If not enabled, this method returns immediately without doing anything
+     * @param metaConnection
+     * @throws SQLException
+     */
+    private void createSchemaIfNotExistsSystemNSMappingEnabled(PhoenixConnection metaConnection) throws SQLException {
+        // HBase Namespace SYSTEM is assumed to be already created inside {@link ensureTableCreated(byte[], PTableType,
+        // Map<String, Object>, List<Pair<byte[], Map<String, Object>>>, byte[][], boolean, boolean, boolean)}.
+        // This statement will create an entry for the SYSTEM namespace in the SYSCAT table, so that GRANT/REVOKE
+        // commands can work with SYSTEM Namespace. (See PHOENIX-4227 https://issues.apache.org/jira/browse/PHOENIX-4227)
+        if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
+          ConnectionQueryServicesImpl.this.getProps())) {
+            try {
+                metaConnection.createStatement().execute("CREATE SCHEMA IF NOT EXISTS "
+                  + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
+            } catch (NewerSchemaAlreadyExistsException e) {
+                // Older clients with appropriate perms may try getting a new connection
+                // This results in NewerSchemaAlreadyExistsException, so we can safely ignore it here
+            } catch (PhoenixIOException e) {
+                if (!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), AccessDeniedException.class))) {
+                    // Ignore ADE
+                } else {
+                    throw e;
+                }
+            }
+        }
+    }
+
+    /**
+     * Upgrade the SYSCAT schema if required
+     * @param metaConnection
+     * @param currentServerSideTableTimeStamp
+     * @return Phoenix connection object
+     * @throws SQLException
+     * @throws IOException
+     * @throws TimeoutException
+     * @throws InterruptedException
+     */
+    // Available for testing
+    protected PhoenixConnection upgradeSystemCatalogIfRequired(PhoenixConnection metaConnection,
+      long currentServerSideTableTimeStamp) throws SQLException, IOException, TimeoutException, InterruptedException {
+        String columnsToAdd = "";
+        // This will occur if we have an older SYSTEM.CATALOG and we need to update it to
+        // include any new columns we've added.
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
+            // We know that we always need to add the STORE_NULLS column for 4.3 release
+            columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.STORE_NULLS
+              + " " + PBoolean.INSTANCE.getSqlTypeName());
+            try (HBaseAdmin admin = getAdmin()) {
+                HTableDescriptor[] localIndexTables = admin
+                  .listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + ".*");
+                for (HTableDescriptor table : localIndexTables) {
+                    if (table.getValue(MetaDataUtil.PARENT_TABLE_KEY) == null
+                      && table.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME) != null) {
+                        table.setValue(MetaDataUtil.PARENT_TABLE_KEY,
+                          MetaDataUtil.getLocalIndexUserTableName(table.getNameAsString()));
+                        // Explicitly disable, modify and enable the table to ensure
+                        // co-location of data and index regions. If we just modify the
+                        // table descriptor when online schema change enabled may reopen
+                        // the region in same region server instead of following data region.
+                        admin.disableTable(table.getTableName());
+                        admin.modifyTable(table.getTableName(), table);
+                        admin.enableTable(table.getTableName());
+                    }
+                }
+            }
+        }
+
+        // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
+        // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+        // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+        // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+        // the column names that have been added to SYSTEM.CATALOG since 4.0.
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
+            columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " "
+              + PUnsignedTinyint.INSTANCE.getSqlTypeName() + ", "
+              + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " "
+              + PLong.INSTANCE.getSqlTypeName());
+        }
+
+        // If we have some new columns from 4.1-4.3 to add, add them now.
+        if (!columnsToAdd.isEmpty()) {
+            // Ugh..need to assign to another local variable to keep eclipse happy.
+            PhoenixConnection newMetaConnection = addColumnsIfNotExists(metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0, columnsToAdd);
+            metaConnection = newMetaConnection;
+        }
+
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0) {
+            columnsToAdd = PhoenixDatabaseMetaData.BASE_COLUMN_COUNT + " "
+              + PInteger.INSTANCE.getSqlTypeName();
+            try {
+                metaConnection = addColumn(metaConnection,
+                  PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                  MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd,
+                  false);
+                upgradeTo4_5_0(metaConnection);
+            } catch (ColumnAlreadyExistsException ignored) {
+                /*
+                 * Upgrade to 4.5 is a slightly special case. We use the fact that the
+                 * column BASE_COLUMN_COUNT is already part of the meta-data schema as the
+                 * signal that the server side upgrade has finished or is in progress.
+                 */
+                logger.debug("No need to run 4.5 upgrade");
+            }
+            Properties p = PropertiesUtil.deepCopy(metaConnection.getClientInfo());
+            p.remove(PhoenixRuntime.CURRENT_SCN_ATTRIB);
+            p.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
+            PhoenixConnection conn = new PhoenixConnection(
+              ConnectionQueryServicesImpl.this, metaConnection.getURL(), p,
+              metaConnection.getMetaDataCache());
+            try {
+                List<String> tablesNeedingUpgrade = UpgradeUtil
+                  .getPhysicalTablesWithDescRowKey(conn);
+                if (!tablesNeedingUpgrade.isEmpty()) {
+                    logger.warn("The following tables require upgrade due to a bug causing the row key to be incorrect for descending columns and ascending BINARY columns (PHOENIX-2067 and PHOENIX-2120):\n"
+                      + Joiner.on(' ').join(tablesNeedingUpgrade)
+                      + "\nTo upgrade issue the \"bin/psql.py -u\" command.");
+                }
+                List<String> unsupportedTables = UpgradeUtil
+                  .getPhysicalTablesWithDescVarbinaryRowKey(conn);
+                if (!unsupportedTables.isEmpty()) {
+                    logger.warn("The following tables use an unsupported VARBINARY DESC construct and need to be changed:\n"
+                      + Joiner.on(' ').join(unsupportedTables));
+                }
+            } catch (Exception ex) {
+                logger.error(
+                  "Unable to determine tables requiring upgrade due to PHOENIX-2067",
+                  ex);
+            } finally {
+                conn.close();
+            }
+        }
+        // Add these columns one at a time, each with different timestamps so that if folks
+        // have
+        // run the upgrade code already for a snapshot, we'll still enter this block (and do
+        // the
+        // parts we haven't yet done).
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0) {
+            columnsToAdd = PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP + " "
+              + PBoolean.INSTANCE.getSqlTypeName();
+            metaConnection = addColumnsIfNotExists(metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, columnsToAdd);
+        }
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
+            // Drop old stats table so that new stats table is created
+            metaConnection = dropStatsTable(metaConnection,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
+              PhoenixDatabaseMetaData.TRANSACTIONAL + " "
+                + PBoolean.INSTANCE.getSqlTypeName());
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
+              PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " "
+                + PLong.INSTANCE.getSqlTypeName());
+            metaConnection = setImmutableTableIndexesImmutable(metaConnection,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
+            metaConnection = updateSystemCatalogTimestamp(metaConnection,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+            ConnectionQueryServicesImpl.this.removeTable(null,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+            clearCache();
+        }
+
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0) {
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 2,
+              PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED + " "
+                + PBoolean.INSTANCE.getSqlTypeName());
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 1,
+              PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ + " "
+                + PVarchar.INSTANCE.getSqlTypeName());
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0,
+              PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA + " "
+                + PBoolean.INSTANCE.getSqlTypeName());
+            metaConnection = UpgradeUtil.disableViewIndexes(metaConnection);
+            if (getProps().getBoolean(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB,
+              QueryServicesOptions.DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)) {
+                metaConnection = UpgradeUtil.upgradeLocalIndexes(metaConnection);
+            }
+            ConnectionQueryServicesImpl.this.removeTable(null,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0);
+            clearCache();
+        }
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0) {
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0,
+              PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + " "
+                + PLong.INSTANCE.getSqlTypeName());
+            ConnectionQueryServicesImpl.this.removeTable(null,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0);
+            clearCache();
+        }
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0) {
+            metaConnection = addColumnQualifierColumn(metaConnection, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 3);
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 2,
+              PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME + " "
+                + PTinyint.INSTANCE.getSqlTypeName());
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 1,
+              PhoenixDatabaseMetaData.ENCODING_SCHEME + " "
+                + PTinyint.INSTANCE.getSqlTypeName());
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0,
+              PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER + " "
+                + PInteger.INSTANCE.getSqlTypeName());
+            ConnectionQueryServicesImpl.this.removeTable(null,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0);
+            clearCache();
+        }
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
+            metaConnection = addColumnsIfNotExists(
+              metaConnection,
+              PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+              MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0,
+              PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION + " "
+                + 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());
+        }
+        return metaConnection;
+    }
+
+    /**
      * There is no other locking needed here since only one connection (on the same or different JVM) will be able to
      * acquire the upgrade mutex via {@link #acquireUpgradeMutex(long, byte[])}.
      */
@@ -2674,244 +2976,54 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             metaConnection = new PhoenixConnection(ConnectionQueryServicesImpl.this, globalUrl,
                     scnProps, newEmptyMetaData());
             metaConnection.setRunningUpgrade(true);
+            // Always try to create SYSTEM.MUTEX table since we need it to acquire the upgrade mutex.
+            // Upgrade or migration is not possible without the upgrade mutex
+            try (HBaseAdmin admin = getAdmin()) {
+                createSysMutexTableIfNotExists(admin);
+            }
             try {
                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
             } catch (NewerTableAlreadyExistsException ignore) {
                 // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed
                 // timestamp. A TableAlreadyExistsException is not thrown, since the table only exists
                 // *after* this fixed timestamp.
+            } catch (UpgradeRequiredException e) {
+                // This is thrown while trying to create SYSTEM:CATALOG to indicate that we must migrate SYSTEM tables
+                // to the SYSTEM namespace and/or upgrade SYSCAT if required
+                sysCatalogTableName = SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getNameAsString();
+                if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, ConnectionQueryServicesImpl.this.getProps())) {
+                    // Try acquiring a lock in SYSMUTEX table before migrating the tables since it involves disabling the table.
+                    if (acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP, mutexRowKey)) {
+                        logger.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace "
+                          + "and/or upgrading " + sysCatalogTableName);
+                    }
+                    // We will not reach here if we fail to acquire the lock, since it throws UpgradeInProgressException
+
+                    // If SYSTEM tables exist, they are migrated to HBase SYSTEM namespace
+                    // If they don't exist or they're already migrated, this method will return immediately
+                    ensureSystemTablesMigratedToSystemNamespace();
+                    logger.debug("Migrated SYSTEM tables to SYSTEM namespace");
+                    metaConnection = upgradeSystemCatalogIfRequired(metaConnection, e.getSystemCatalogTimeStamp());
+                }
             } catch (TableAlreadyExistsException e) {
                 long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                 sysCatalogTableName = e.getTable().getPhysicalName().getString();
                 if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
-                    if (currentServerSideTableTimeStamp <= MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0) {
-                        try (HBaseAdmin admin = getAdmin()) {
-                            createSysMutexTableIfNotExists(admin, this.getProps());
-                        }
-                    }
+                    // Try acquiring a lock in SYSMUTEX table before upgrading SYSCAT. If we cannot acquire the lock,
+                    // it means some old client is either migrating SYSTEM tables or trying to upgrade the schema of
+                    // SYSCAT table and hence it should not be interrupted
                     if (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp, mutexRowKey)) {
+                        logger.debug("Acquired lock in SYSMUTEX table for upgrading " + sysCatalogTableName);
                         snapshotName = getSysCatalogSnapshotName(currentServerSideTableTimeStamp);
                         createSnapshot(snapshotName, sysCatalogTableName);
                         snapshotCreated = true;
+                        logger.debug("Created snapshot for SYSCAT");
                     }
+                    // We will not reach here if we fail to acquire the lock, since it throws UpgradeInProgressException
                 }
-                String columnsToAdd = "";
-                // This will occur if we have an older SYSTEM.CATALOG and we need to update it to
-                // include any new columns we've added.
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
-                    // We know that we always need to add the STORE_NULLS column for 4.3 release
-                    columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.STORE_NULLS
-                            + " " + PBoolean.INSTANCE.getSqlTypeName());
-                    try (HBaseAdmin admin = getAdmin()) {
-                        HTableDescriptor[] localIndexTables = admin
-                                .listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + ".*");
-                        for (HTableDescriptor table : localIndexTables) {
-                            if (table.getValue(MetaDataUtil.PARENT_TABLE_KEY) == null
-                                    && table.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME) != null) {
-                                table.setValue(MetaDataUtil.PARENT_TABLE_KEY,
-                                        MetaDataUtil.getLocalIndexUserTableName(table.getNameAsString()));
-                                // Explicitly disable, modify and enable the table to ensure
-                                // co-location of data and index regions. If we just modify the
-                                // table descriptor when online schema change enabled may reopen 
-                                // the region in same region server instead of following data region.
-                                admin.disableTable(table.getTableName());
-                                admin.modifyTable(table.getTableName(), table);
-                                admin.enableTable(table.getTableName());
-                            }
-                        }
-                    }
-                }
-
-                // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
-                // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
-                // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
-                // the column names that have been added to SYSTEM.CATALOG since 4.0.
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
-                    columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " "
-                            + PUnsignedTinyint.INSTANCE.getSqlTypeName() + ", "
-                            + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " "
-                            + PLong.INSTANCE.getSqlTypeName());
-                }
-
-                // If we have some new columns from 4.1-4.3 to add, add them now.
-                if (!columnsToAdd.isEmpty()) {
-                    // Ugh..need to assign to another local variable to keep eclipse happy.
-                    PhoenixConnection newMetaConnection = addColumnsIfNotExists(metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0, columnsToAdd);
-                    metaConnection = newMetaConnection;
-                }
-
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0) {
-                    columnsToAdd = PhoenixDatabaseMetaData.BASE_COLUMN_COUNT + " "
-                            + PInteger.INSTANCE.getSqlTypeName();
-                    try {
-                        metaConnection = addColumn(metaConnection,
-                                PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd,
-                                false);
-                        upgradeTo4_5_0(metaConnection);
-                    } catch (ColumnAlreadyExistsException ignored) {
-                        /*
-                         * Upgrade to 4.5 is a slightly special case. We use the fact that the
-                         * column BASE_COLUMN_COUNT is already part of the meta-data schema as the
-                         * signal that the server side upgrade has finished or is in progress.
-                         */
-                        logger.debug("No need to run 4.5 upgrade");
-                    }
-                    Properties p = PropertiesUtil.deepCopy(metaConnection.getClientInfo());
-                    p.remove(PhoenixRuntime.CURRENT_SCN_ATTRIB);
-                    p.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
-                    PhoenixConnection conn = new PhoenixConnection(
-                            ConnectionQueryServicesImpl.this, metaConnection.getURL(), p,
-                            metaConnection.getMetaDataCache());
-                    try {
-                        List<String> tablesNeedingUpgrade = UpgradeUtil
-                                .getPhysicalTablesWithDescRowKey(conn);
-                        if (!tablesNeedingUpgrade.isEmpty()) {
-                            logger.warn("The following tables require upgrade due to a bug causing the row key to be incorrect for descending columns and ascending BINARY columns (PHOENIX-2067 and PHOENIX-2120):\n"
-                                    + Joiner.on(' ').join(tablesNeedingUpgrade)
-                                    + "\nTo upgrade issue the \"bin/psql.py -u\" command.");
-                        }
-                        List<String> unsupportedTables = UpgradeUtil
-                                .getPhysicalTablesWithDescVarbinaryRowKey(conn);
-                        if (!unsupportedTables.isEmpty()) {
-                            logger.warn("The following tables use an unsupported VARBINARY DESC construct and need to be changed:\n"
-                                    + Joiner.on(' ').join(unsupportedTables));
-                        }
-                    } catch (Exception ex) {
-                        logger.error(
-                                "Unable to determine tables requiring upgrade due to PHOENIX-2067",
-                                ex);
-                    } finally {
-                        conn.close();
-                    }
-                }
-                // Add these columns one at a time, each with different timestamps so that if folks
-                // have
-                // run the upgrade code already for a snapshot, we'll still enter this block (and do
-                // the
-                // parts we haven't yet done).
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0) {
-                    columnsToAdd = PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP + " "
-                            + PBoolean.INSTANCE.getSqlTypeName();
-                    metaConnection = addColumnsIfNotExists(metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, columnsToAdd);
-                }
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
-                    // Drop old stats table so that new stats table is created
-                    metaConnection = dropStatsTable(metaConnection,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
-                            PhoenixDatabaseMetaData.TRANSACTIONAL + " "
-                                    + PBoolean.INSTANCE.getSqlTypeName());
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
-                            PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " "
-                                    + PLong.INSTANCE.getSqlTypeName());
-                    metaConnection = setImmutableTableIndexesImmutable(metaConnection,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                    metaConnection = updateSystemCatalogTimestamp(metaConnection,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
-                    ConnectionQueryServicesImpl.this.removeTable(null,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
-                    clearCache();
-                }
-
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0) {
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 2,
-                            PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED + " "
-                                    + PBoolean.INSTANCE.getSqlTypeName());
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 1,
-                            PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ + " "
-                                    + PVarchar.INSTANCE.getSqlTypeName());
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0,
-                            PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA + " "
-                                    + PBoolean.INSTANCE.getSqlTypeName());
-                    metaConnection = UpgradeUtil.disableViewIndexes(metaConnection);
-                    if (getProps().getBoolean(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB,
-                            QueryServicesOptions.DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)) {
-                        metaConnection = UpgradeUtil.upgradeLocalIndexes(metaConnection);
-                    }
-                    ConnectionQueryServicesImpl.this.removeTable(null,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0);
-                    clearCache();
-                }
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0) {
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0,
-                            PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + " "
-                                    + PLong.INSTANCE.getSqlTypeName());
-                    ConnectionQueryServicesImpl.this.removeTable(null,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0);
-                    clearCache();
-                }
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0) {
-                    metaConnection = addColumnQualifierColumn(metaConnection, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 3);
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 2,
-                            PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME + " "
-                                    + PTinyint.INSTANCE.getSqlTypeName());
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 1,
-                            PhoenixDatabaseMetaData.ENCODING_SCHEME + " "
-                                    + PTinyint.INSTANCE.getSqlTypeName());
-                    metaConnection = addColumnsIfNotExists(
-                            metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0,
-                            PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER + " "
-                                    + PInteger.INSTANCE.getSqlTypeName());
-                    ConnectionQueryServicesImpl.this.removeTable(null,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
-                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0);
-                    clearCache();
-                }
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
-                    metaConnection = addColumnsIfNotExists(
-                        metaConnection,
-                        PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                        MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0,
-                        PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION + " "
-                                + 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());
-                }
+                metaConnection = upgradeSystemCatalogIfRequired(metaConnection, currentServerSideTableTimeStamp);
             }
 
-
             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(
                     QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
                     QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
@@ -2997,6 +3109,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
+
+            // In case namespace mapping is enabled and system table to system namespace mapping is also enabled,
+            // create an entry for the SYSTEM namespace in the SYSCAT table, so that GRANT/REVOKE commands can work
+            // with SYSTEM Namespace
+            createSchemaIfNotExistsSystemNSMappingEnabled(metaConnection);
+
             ConnectionQueryServicesImpl.this.upgradeRequired.set(false);
             success = true;
         } catch (UpgradeInProgressException | UpgradeNotRequiredException e) {
@@ -3218,28 +3336,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
-    void ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps props)
+    void ensureSystemTablesMigratedToSystemNamespace()
             throws SQLException, IOException, IllegalArgumentException, InterruptedException {
-        if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }
-
-        boolean acquiredMutexLock = false;
-        byte[] mutexRowKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
-                PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE);
+        if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, this.getProps())) { return; }
 
         HTableInterface metatable = null;
         try (HBaseAdmin admin = getAdmin()) {
-             // SYSTEM namespace needs to be created via HBase API's because "CREATE SCHEMA" statement tries to write its metadata
-             // in SYSTEM:CATALOG table. Without SYSTEM namespace, SYSTEM:CATALOG table cannot be created.
-            try {
-                ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
-            } catch (PhoenixIOException e) {
-                // We could either:
-                // 1) Not access the NS descriptor. The NS may or may not exist at this point.
-                // 2) We could not create the NS
-                // Regardless of the case 1 or 2, if the NS does not exist, we will error expectedly
-                // below. If the NS does exist and is mapped, the below check will exit gracefully.
-            }
-
             List<TableName> tableNames = getSystemTableNamesInDefaultNamespace(admin);
             // No tables exist matching "SYSTEM\..*", they are all already in "SYSTEM:.*"
             if (tableNames.size() == 0) { return; }
@@ -3248,33 +3350,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 logger.warn("Expected 5 system tables but found " + tableNames.size() + ":" + tableNames);
             }
 
-            // Try acquiring a lock in SYSMUTEX table before migrating the tables since it involves disabling the table
-            // If we cannot acquire lock, it means some old client is either migrating SYSCAT or trying to upgrade the
-            // schema of SYSCAT table and hence it should not be interrupted
-            // Create mutex if not already created
-            createSysMutexTableIfNotExists(admin, props);
-            acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP, mutexRowKey);
-            if(acquiredMutexLock) {
-                logger.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace");
-            }
-            // We will not reach here if we fail to acquire the lock, since it throws UpgradeInProgressException
-
             // Handle the upgrade of SYSMUTEX table separately since it doesn't have any entries in SYSCAT
             logger.info("Migrating SYSTEM.MUTEX table to SYSTEM namespace.");
             String sysMutexSrcTableName = PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME;
-            String sysMutexDestTableName = SchemaUtil.getPhysicalName(sysMutexSrcTableName.getBytes(), props).getNameAsString();
+            String sysMutexDestTableName = SchemaUtil.getPhysicalName(sysMutexSrcTableName.getBytes(), this.getProps()).getNameAsString();
             UpgradeUtil.mapTableToNamespace(admin, sysMutexSrcTableName, sysMutexDestTableName, PTableType.SYSTEM);
             tableNames.remove(PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME);
 
             byte[] mappedSystemTable = SchemaUtil
-                    .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props).getName();
+                    .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName();
             metatable = getTable(mappedSystemTable);
             if (tableNames.contains(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)) {
                 if (!admin.tableExists(mappedSystemTable)) {
                     logger.info("Migrating SYSTEM.CATALOG table to SYSTEM namespace.");
                     // Actual migration of SYSCAT table
                     UpgradeUtil.mapTableToNamespace(admin, metatable,
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, props, null, PTableType.SYSTEM,
+                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, this.getProps(), null, PTableType.SYSTEM,
                             null);
                     // Invalidate the client-side metadataCache
                     ConnectionQueryServicesImpl.this.removeTable(null,
@@ -3285,7 +3376,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
             for (TableName table : tableNames) {
                 logger.info(String.format("Migrating %s table to SYSTEM namespace.", table.getNameAsString()));
-                UpgradeUtil.mapTableToNamespace(admin, metatable, table.getNameAsString(), props, null, PTableType.SYSTEM,
+                UpgradeUtil.mapTableToNamespace(admin, metatable, table.getNameAsString(), this.getProps(), null, PTableType.SYSTEM,
                         null);
                 ConnectionQueryServicesImpl.this.removeTable(null, table.getNameAsString(), null,
                         MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0);
@@ -3297,9 +3388,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (metatable != null) {
                 metatable.close();
             }
-            if(acquiredMutexLock) {
-                releaseUpgradeMutex(mutexRowKey);
-            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 aa8209d..14abd63 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
@@ -242,7 +242,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     @Override
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType,
             Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException {
+            boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet) throws SQLException {
         if (tableType == PTableType.INDEX && IndexUtil.isLocalIndexFamily(Bytes.toString(families.iterator().next().getFirst()))) {
             Object dataTableName = tableProps.get(PhoenixDatabaseMetaData.DATA_TABLE_NAME);
             List<HRegionLocation> regionLocations = tableSplits.get(dataTableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 ed9b9da..0d0df37 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
@@ -114,9 +114,9 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     @Override
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType,
             Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException {
+            boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet) throws SQLException {
         return getDelegate().createTable(tableMetaData, physicalName, tableType, tableProps, families, splits,
-                isNamespaceMapped, allocateIndexId);
+                isNamespaceMapped, allocateIndexId, isDoNotUpgradePropSet);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/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 1fb668e..b15072a 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
@@ -2694,7 +2694,8 @@ public class MetaDataClient {
             MetaDataMutationResult result = connection.getQueryServices().createTable(
                     tableMetaData,
                     viewType == ViewType.MAPPED || allocateIndexId ? physicalNames.get(0).getBytes() : null,
-                    tableType, tableProps, familyPropList, splits, isNamespaceMapped, allocateIndexId);
+                    tableType, tableProps, familyPropList, splits, isNamespaceMapped, allocateIndexId,
+                    UpgradeUtil.isNoUpgradeSet(connection.getClientInfo()));
             MutationCode code = result.getMutationCode();
             switch(code) {
             case TABLE_ALREADY_EXISTS:

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
index b5c3e4a..46907d9 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
@@ -47,9 +47,9 @@ public class ConnectionQueryServicesImplTest {
         ConnectionQueryServicesImpl cqs = mock(ConnectionQueryServicesImpl.class);
         // Invoke the real methods for these two calls
         when(cqs.createSchema(any(List.class), anyString())).thenCallRealMethod();
-        doCallRealMethod().when(cqs).ensureSystemTablesMigratedToSystemNamespace(any(ReadOnlyProps.class));
+        doCallRealMethod().when(cqs).ensureSystemTablesMigratedToSystemNamespace();
         // Do nothing for this method, just check that it was invoked later
-        doNothing().when(cqs).createSysMutexTableIfNotExists(any(HBaseAdmin.class), any(ReadOnlyProps.class));
+        doNothing().when(cqs).createSysMutexTableIfNotExists(any(HBaseAdmin.class));
 
         // Spoof out this call so that ensureSystemTablesUpgrade() will return-fast.
         when(cqs.getSystemTableNamesInDefaultNamespace(any(HBaseAdmin.class))).thenReturn(Collections.<TableName> emptyList());
@@ -60,7 +60,8 @@ public class ConnectionQueryServicesImplTest {
         // Make sure that ensureSystemTablesMigratedToSystemNamespace will try to migrate the system tables.
         Map<String,String> props = new HashMap<>();
         props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
-        cqs.ensureSystemTablesMigratedToSystemNamespace(new ReadOnlyProps(props));
+        when(cqs.getProps()).thenReturn(new ReadOnlyProps(props));
+        cqs.ensureSystemTablesMigratedToSystemNamespace();
 
         // Should be called after upgradeSystemTables()
         // Proves that execution proceeded

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8cda8141/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 2ba2b4c..369522c 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -168,6 +168,7 @@ message GetVersionRequest {
 
 message GetVersionResponse {
   required int64 version = 1;
+  optional int64 systemCatalogTimestamp = 2;
 }
 
 message ClearTableFromCacheRequest {


[05/21] phoenix git commit: PHOENIX-2715 Query Log (addendum)

Posted by pb...@apache.org.
PHOENIX-2715 Query Log (addendum)


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

Branch: refs/heads/4.x-cdh5.12
Commit: 6e899e50c090e40934d6f7ad2e65aa9eef616766
Parents: b291068
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Apr 10 11:12:38 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:26:26 2018 +0100

----------------------------------------------------------------------
 .../MigrateSystemTablesToSystemNamespaceIT.java | 40 +++++++++++---------
 1 file changed, 22 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6e899e50/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 0d64ca9..d20ffa9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -16,6 +16,26 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -39,31 +59,15 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-
 @Category(NeedsOwnMiniClusterTest.class)
 public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
 
     private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
             "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-            "SYSTEM.MUTEX"));
+            "SYSTEM.MUTEX","SYSTEM.LOG"));
     private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
             Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
-                    "SYSTEM:MUTEX"));
+                    "SYSTEM:MUTEX","SYSTEM:LOG"));
     private static final String SCHEMA_NAME = "MIGRATETEST";
     private static final String TABLE_NAME =
             SCHEMA_NAME + "." + MigrateSystemTablesToSystemNamespaceIT.class.getSimpleName().toUpperCase();