You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by an...@apache.org on 2016/05/16 06:47:21 UTC

[1/3] phoenix git commit: PHOENIX-2884 Fix EXPLAIN plan for offset on salted table

Repository: phoenix
Updated Branches:
  refs/heads/master 98e783cf6 -> 5d641ad34


PHOENIX-2884 Fix EXPLAIN plan for offset on salted table


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

Branch: refs/heads/master
Commit: 33fb201ebe46ba90f81658ef9b487f3b74070a4a
Parents: 98e783c
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon May 16 12:16:21 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon May 16 12:16:21 2016 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/QueryWithOffsetIT.java      | 25 ++++++++++++++++----
 .../phoenix/iterate/BaseResultIterators.java    |  4 +---
 2 files changed, 21 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/33fb201e/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
index dd21f31..c5b6959 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
@@ -38,6 +38,7 @@ import java.util.Properties;
 
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,16 +51,18 @@ import com.google.common.collect.Maps;
 @RunWith(Parameterized.class)
 public class QueryWithOffsetIT extends BaseOwnClusterHBaseManagedTimeIT {
     
-    private String tableName;
+    private String tableName = "T";
     private final String[] strings = { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p",
             "q", "r", "s", "t", "u", "v", "w", "x", "y", "z" };
     private final String ddl;
+    private final boolean isSalted;
 
     public QueryWithOffsetIT(String preSplit) {
         this.tableName=tableName + "_" + preSplit.charAt(2);
         this.ddl = "CREATE TABLE " + tableName + " (t_id VARCHAR NOT NULL,\n" + "k1 INTEGER NOT NULL,\n"
                 + "k2 INTEGER NOT NULL,\n" + "C3.k3 INTEGER,\n" + "C2.v1 VARCHAR,\n"
                 + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) " + preSplit;
+        this.isSalted = preSplit.startsWith(" SALT_BUCKETS");
     }
 
     @BeforeClass
@@ -122,16 +125,28 @@ public class QueryWithOffsetIT extends BaseOwnClusterHBaseManagedTimeIT {
         updateStatistics(conn);
         String query = "SELECT t_id from " + tableName + " offset " + offset;
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-        rs.next();
-        rs.next();
-        rs.next();
-        assertEquals("    SERVER OFFSET " + offset, rs.getString(1));
+        if(!isSalted){
+            assertEquals("CLIENT SERIAL 1-WAY FULL SCAN OVER T_P\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                    + "    SERVER OFFSET " + offset, QueryUtil.getExplainPlan(rs));
+        }else{
+            assertEquals("CLIENT PARALLEL 10-WAY FULL SCAN OVER T_A\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                    + "CLIENT MERGE SORT\n" + "CLIENT OFFSET " + offset, QueryUtil.getExplainPlan(rs));
+        }
         rs = conn.createStatement().executeQuery(query);
         int i = 0;
         while (i++ < strings.length - offset) {
             assertTrue(rs.next());
             assertEquals(strings[offset + i - 1], rs.getString(1));
         }
+        query = "SELECT t_id from " + tableName + " ORDER BY v1 offset " + offset;
+        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+        if (!isSalted) {
+            assertEquals("CLIENT PARALLEL 5-WAY FULL SCAN OVER T_P\n" + "    SERVER SORTED BY [C2.V1]\n"
+                    + "CLIENT MERGE SORT\n" + "CLIENT OFFSET " + offset, QueryUtil.getExplainPlan(rs));
+        } else {
+            assertEquals("CLIENT PARALLEL 10-WAY FULL SCAN OVER T_A\n" + "    SERVER SORTED BY [C2.V1]\n"
+                    + "CLIENT MERGE SORT\n" + "CLIENT OFFSET " + offset, QueryUtil.getExplainPlan(rs));
+        }
         conn.close();
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33fb201e/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index bee2201..4f49e2d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.iterate;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_ACTUAL_START_ROW;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_START_ROW_SUFFIX;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_STOP_ROW_SUFFIX;
-
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QUERY_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
@@ -64,7 +63,6 @@ import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
-import org.apache.phoenix.execute.ScanPlan;
 import org.apache.phoenix.filter.ColumnProjectionFilter;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
@@ -334,7 +332,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
     
     public BaseResultIterators(QueryPlan plan, Integer perScanLimit, Integer offset, ParallelScanGrouper scanGrouper, Scan scan) throws SQLException {
         super(plan.getContext(), plan.getTableRef(), plan.getGroupBy(), plan.getOrderBy(),
-                plan.getStatement().getHint(), plan.getLimit(), plan instanceof ScanPlan ? plan.getOffset() : null);
+                plan.getStatement().getHint(), plan.getLimit(), offset);
         this.plan = plan;
         this.scan = scan;
         this.scanGrouper = scanGrouper;


[2/3] phoenix git commit: PHOENIX-2862 Do client server compatibility checks before upgrading system tables

Posted by an...@apache.org.
PHOENIX-2862 Do client server compatibility checks before upgrading system tables


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

Branch: refs/heads/master
Commit: 20ce95b0c09d7aaa394f2e703f6b1e193141ad33
Parents: 33fb201
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon May 16 12:16:47 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon May 16 12:16:47 2016 +0530

----------------------------------------------------------------------
 .../end2end/QueryDatabaseMetaDataIT.java        |  1 +
 .../org/apache/phoenix/end2end/UpgradeIT.java   |  4 +-
 .../apache/phoenix/compile/FromCompiler.java    |  3 +
 .../coprocessor/MetaDataEndpointImpl.java       | 30 ++++---
 .../coprocessor/MetaDataRegionObserver.java     |  9 +-
 .../phoenix/exception/SQLExceptionCode.java     |  8 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  4 +-
 .../query/ConnectionQueryServicesImpl.java      | 86 ++++++++++++++------
 .../org/apache/phoenix/query/QueryServices.java |  4 +-
 .../phoenix/query/QueryServicesOptions.java     |  2 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   | 46 ++++++++---
 .../org/apache/phoenix/util/SchemaUtil.java     | 15 +++-
 .../org/apache/phoenix/util/UpgradeUtil.java    | 48 ++++++-----
 .../apache/phoenix/util/MetaDataUtilTest.java   | 36 ++++++++
 14 files changed, 216 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 1d0f767..fbea0f0 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
@@ -147,6 +147,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
         assertEquals(CUSTOM_ENTITY_DATA_NAME, rs.getString("TABLE_NAME"));
         assertEquals(PTableType.TABLE.toString(), rs.getString("TABLE_TYPE"));
         assertEquals("false", rs.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
+        assertEquals(Boolean.FALSE, rs.getBoolean(PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED));
 
         rs = dbmd.getTables(null, CUSTOM_ENTITY_DATA_SCHEMA_NAME, CUSTOM_ENTITY_DATA_NAME, null);
         assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index 1c86039..37d285f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@ -129,7 +129,7 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             String indexName = "IDX";
             String localIndexName = "LIDX";
             String[] tableNames = new String[] { phoenixFullTableName, schemaName + "." + indexName,
-                    schemaName + "." + localIndexName, "diff.v", "test.v" };
+                    schemaName + "." + localIndexName, "diff.v", "test.v","v"};
             conn.createStatement().execute("CREATE TABLE " + phoenixFullTableName
                     + "(k VARCHAR PRIMARY KEY, v INTEGER, f INTEGER, g INTEGER NULL, h INTEGER NULL)");
             PreparedStatement upsertStmt = conn
@@ -150,6 +150,7 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             conn.createStatement().execute("CREATE VIEW diff.v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // creating view in schema 'test'
             conn.createStatement().execute("CREATE VIEW test.v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // Creating index on views
             conn.createStatement().execute("create index v_idx on diff.v(col)");
             conn.createStatement().execute("create index v_idx on test.v(col)");
@@ -170,6 +171,7 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             assertTrue(admin.tableExists(MetaDataUtil.getViewIndexPhysicalName(Bytes.toBytes(phoenixFullTableName))));
             Properties props = new Properties();
             props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
+            props.setProperty(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, Boolean.toString(false));
             admin.close();
             PhoenixConnection phxConn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
             UpgradeUtil.upgradeTable(phxConn, phoenixFullTableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 ca0a6c3..ddf9c0c 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
@@ -155,6 +155,9 @@ public class FromCompiler {
                 schemaName = statement.getTableName().getSchemaName();
                 if (schemaName != null) {
                     new SchemaResolver(connection, statement.getTableName().getSchemaName(), true);
+                } else if (connection.getSchema() != null) {
+                    // To ensure schema set through properties or connection string exists before creating table
+                    new SchemaResolver(connection, connection.getSchema(), true);
                 }
             }
             return EMPTY_TABLE_RESOLVER;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 7f222bb..5becf24 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
@@ -92,10 +92,12 @@ import java.util.List;
 import java.util.NavigableMap;
 import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
@@ -159,7 +161,6 @@ import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisit
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
-import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -216,6 +217,7 @@ import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.UpgradeUtil;
@@ -3152,19 +3154,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     @Override
-    public void getVersion(RpcController controller, GetVersionRequest request,
-            RpcCallback<GetVersionResponse> done) {
+    public void getVersion(RpcController controller, GetVersionRequest request, RpcCallback<GetVersionResponse> done) {
 
         GetVersionResponse.Builder builder = GetVersionResponse.newBuilder();
-        // The first 3 bytes of the long is used to encoding the HBase version as major.minor.patch.
-        // The next 4 bytes of the value is used to encode the Phoenix version as major.minor.patch.
-        long version = MetaDataUtil.encodeHBaseAndPhoenixVersions(this.env.getHBaseVersion());
-
-        // The last byte is used to communicate whether or not mutable secondary indexing
-        // was configured properly.
-        version =
-                MetaDataUtil.encodeHasIndexWALCodec(version,
-                    IndexManagementUtil.isWALEditCodecSet(this.env.getConfiguration()));
+        Configuration config = env.getConfiguration();
+        boolean isTablesMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
+                new ReadOnlyProps(config.iterator()));
+        if (isTablesMappingEnabled
+                && PhoenixDatabaseMetaData.MIN_NAMESPACE_MAPPED_PHOENIX_VERSION > request.getClientVersion()) {
+            logger.error("Old client is not compatible when" + " system tables are upgraded to map to namespace");
+            ProtobufUtil.setControllerException(controller,
+                    ServerUtil.createIOException(
+                            SchemaUtil.getPhysicalHBaseTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME,
+                                    isTablesMappingEnabled, PTableType.SYSTEM).getString(),
+                    new DoNotRetryIOException(
+                            "Old client is not compatible when" + " system tables are upgraded to map to namespace")));
+        }
+        long version = MetaDataUtil.encodeVersion(env.getHBaseVersion(), config);
 
         builder.setVersion(version);
         done.run(builder.build());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 39532cd..5243154 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
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -75,6 +74,7 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.UpgradeUtil;
 
@@ -131,9 +131,12 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                 HTableInterface metaTable = null;
                 HTableInterface statsTable = null;
                 try {
+                    ReadOnlyProps props=new ReadOnlyProps(env.getConfiguration().iterator());
                     Thread.sleep(1000);
-                    metaTable = env.getTable(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME));
-                    statsTable = env.getTable(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME));
+                    metaTable = env.getTable(
+                            SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props));
+                    statsTable = env.getTable(
+                            SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES, props));
                     if (UpgradeUtil.truncateStats(metaTable, statsTable)) {
                         LOG.info("Stats are successfully truncated for upgrade 4.7!!");
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 8064ce1..43965f5 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
@@ -402,10 +402,10 @@ public enum SQLExceptionCode {
             return new SchemaNotFoundException(info.getSchemaName());
         }
     }), CANNOT_MUTATE_SCHEMA(723, "43M06", "Cannot mutate schema as schema has existing tables"), SCHEMA_NOT_ALLOWED(
-            724, "43M07",
-            "Schema name not allowed!!"), CREATE_SCHEMA_NOT_ALLOWED(725, "43M08", "Cannot create schema because config "
-                    + QueryServices.IS_NAMESPACE_MAPPING_ENABLED + " for enabling name space mapping isn't enabled.");
-
+            724, "43M07", "Schema name not allowed!!"), CREATE_SCHEMA_NOT_ALLOWED(725, "43M08",
+                    "Cannot create schema because config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " for enabling name space mapping isn't enabled."), INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES(
+                                    726, "43M10", " Inconsistent namespace mapping properites..");
 
     private final int errorCode;
     private final String sqlState;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 344fc3e..ac9d7a6 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
@@ -305,6 +305,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final int MAX_LOCAL_SI_VERSION_DISALLOW = VersionUtil.encodeVersion("0", "98", "8");
     public static final int MIN_LOCAL_SI_VERSION_DISALLOW = VersionUtil.encodeVersion("0", "98", "6");
     public static final int MIN_RENEW_LEASE_VERSION = VersionUtil.encodeVersion("1", "1", "3");
+    public static final int MIN_NAMESPACE_MAPPED_PHOENIX_VERSION = VersionUtil.encodeVersion("4", "8", "0");
     
     // Version below which we should turn off essential column family.
     public static final int ESSENTIAL_FAMILY_VERSION_THRESHOLD = VersionUtil.encodeVersion("0", "94", "7");
@@ -1047,7 +1048,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
                 VIEW_STATEMENT + "," +
                 SQLViewTypeFunction.NAME + "(" + VIEW_TYPE + ") AS " + VIEW_TYPE + "," +
                 SQLIndexTypeFunction.NAME + "(" + INDEX_TYPE + ") AS " + INDEX_TYPE + "," +
-                TRANSACTIONAL +
+                TRANSACTIONAL + "," +
+                IS_NAMESPACE_MAPPED +
                 " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
                 " where " + COLUMN_NAME + " is null" +
                 " and " + COLUMN_FAMILY + " is null" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 931ecfc..f665cb4 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
@@ -148,6 +148,7 @@ import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.EmptySequenceCacheException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
+import org.apache.phoenix.schema.NewerSchemaAlreadyExistsException;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -160,7 +161,6 @@ import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.SaltingUtil;
-import org.apache.phoenix.schema.SchemaAlreadyExistsException;
 import org.apache.phoenix.schema.Sequence;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceKey;
@@ -976,9 +976,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try (HBaseAdmin admin = getAdmin()) {
             NamespaceDescriptor namespaceDescriptor = null;
             try {
+                namespaceDescriptor = admin.getNamespaceDescriptor(schemaName);
+            } catch (org.apache.hadoop.hbase.NamespaceNotFoundException e) {
+
+            }
+            if (namespaceDescriptor == null) {
                 namespaceDescriptor = NamespaceDescriptor.create(schemaName).build();
                 admin.createNamespace(namespaceDescriptor);
-            } catch (org.apache.hadoop.hbase.NamespaceExistException e) {}
+            }
             return namespaceDescriptor;
         } catch (IOException e) {
             sqlE = ServerUtil.parseServerException(e);
@@ -1045,7 +1050,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return null;
                 }
                 if (isMetaTable) {
-                    checkClientServerCompatibility();
+                    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
@@ -1064,7 +1069,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 return null;
             } else {
                 if (isMetaTable) {
-                    checkClientServerCompatibility();
+                    checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
                 }
 
                 if (!modifyExistingMetaData) {
@@ -1139,13 +1144,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return MetaDataUtil.areClientAndServerCompatible(serverVersion);
     }
 
-    private void checkClientServerCompatibility() throws SQLException {
+    private void checkClientServerCompatibility(byte[] metaTable) throws SQLException {
         StringBuilder buf = new StringBuilder("The following servers require an updated " + QueryConstants.DEFAULT_COPROCESS_PATH + " to be put in the classpath of HBase: ");
         boolean isIncompatible = false;
         int minHBaseVersion = Integer.MAX_VALUE;
+        boolean isTableNamespaceMappingEnabled = false;
         try {
             List<HRegionLocation> locations = this
-                    .getAllTableRegions(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
+                    .getAllTableRegions(metaTable);
             Set<HRegionLocation> serverMap = Sets.newHashSetWithExpectedSize(locations.size());
             TreeMap<byte[], HRegionLocation> regionMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
             List<byte[]> regionKeys = Lists.newArrayListWithExpectedSize(locations.size());
@@ -1158,7 +1164,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
 
             HTableInterface ht = this
-                    .getTable(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
+                    .getTable(metaTable);
             final Map<byte[], Long> results =
                     ht.coprocessorService(MetaDataService.class, null, null, new Batch.Call<MetaDataService,Long>() {
                         @Override
@@ -1177,6 +1183,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     });
             for (Map.Entry<byte[],Long> 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);
+
                 if (!isCompatible(result.getValue())) {
                     isIncompatible = true;
                     HRegionLocation name = regionMap.get(result.getKey());
@@ -1188,6 +1197,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     minHBaseVersion = MetaDataUtil.decodeHBaseVersion(result.getValue());
                 }
             }
+            if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
+                    getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                    .setMessage(
+                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                                                    + " is consitent on client and server.")
+                                    .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
@@ -2328,11 +2344,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
                                     ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
-                            
-                            if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
-                                ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
+                            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);
+                                    }
+                                    ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
+                                } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
+                                                SQLExceptionCode.INCONSISTENET_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 {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
 
@@ -2457,11 +2488,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0);
                                     clearCache();
                                 }
-                                try {
-                                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_SYSTEM_SCHEMA);
-                                } catch (SchemaAlreadyExistsException sa) {
-
-                                }
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
@@ -2528,6 +2554,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
+                            if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
+                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                try {
+                                    metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
+                                            + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
+                                } catch (NewerSchemaAlreadyExistsException e) {}
+                            }
                             scheduleRenewLeaseTasks(); 
                         } catch (Exception e) {
                             if (e instanceof SQLException) {
@@ -2571,22 +2604,27 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (tableNames.size() == 0) { return; }
                         if (tableNames.size() > 4) { throw new IllegalArgumentException(
                                 "Expected 4 system table only but found " + tableNames.size() + ":" + tableNames); }
-                        metatable = getTable(SchemaUtil
-                                .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props).getName());
+                        byte[] mappedSystemTable = SchemaUtil
+                                .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props).getName();
+                        metatable = getTable(mappedSystemTable);
                         if (tableNames.contains(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME)) {
-                            UpgradeUtil.mapTableToNamespace(admin, metatable,
-                                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, props,
-                                    MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0, PTableType.SYSTEM);
+                            if (!admin.tableExists(mappedSystemTable)) {
+                                UpgradeUtil.mapTableToNamespace(admin, metatable,
+                                        PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, props,
+                                        MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0, PTableType.SYSTEM);
+                                ConnectionQueryServicesImpl.this.removeTable(null,
+                                        PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+                                        MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0);
+                            }
                             tableNames.remove(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
-                            ConnectionQueryServicesImpl.this.removeTable(null,
-                                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
-                                    MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0);
                         }
                         for (String table : tableNames) {
                             UpgradeUtil.mapTableToNamespace(admin, metatable, table, props,
                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0, PTableType.SYSTEM);
                             ConnectionQueryServicesImpl.this.removeTable(null, table, null,
                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0);
+                        }
+                        if (!tableNames.isEmpty()) {
                             clearCache();
                         }
                     } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 b4e1e18..b66b67b 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
@@ -51,8 +51,8 @@ public interface QueryServices extends SQLCloseable {
     // consistency configuration setting
     public static final String CONSISTENCY_ATTRIB = "phoenix.connection.consistency";
     public static final String SCHEMA_ATTRIB = "phoenix.connection.schema";
-    public static final String IS_NAMESPACE_MAPPING_ENABLED  = "phoenix.connection.isNamespaceMappingEnabled";
-    public static final String IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE  = "phoenix.connection.mapSystemTablesToNamespace";
+    public static final String IS_NAMESPACE_MAPPING_ENABLED  = "phoenix.schema.isNamespaceMappingEnabled";
+    public static final String IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE  = "phoenix.schema.mapSystemTablesToNamespace";
     // joni byte regex engine setting
     public static final String USE_BYTE_BASED_REGEX_ATTRIB = "phoenix.regex.byteBased";
     public static final String DRIVER_SHUTDOWN_TIMEOUT_MS = "phoenix.shutdown.timeoutMs";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 4280b50..a7c91f6 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
@@ -136,7 +136,7 @@ public class QueryServicesOptions {
     // latency and less parallelization.
     public static final long DEFAULT_SCAN_RESULT_CHUNK_SIZE = 2999;
     public static final boolean DEFAULT_IS_NAMESPACE_MAPPING_ENABLED = false;
-    public static final boolean DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE = false;
+    public static final boolean DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE = true;
 
     //
     // Spillable GroupBy - SPGBY prefix

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 995f44b..116b62b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -116,7 +117,7 @@ public class MetaDataUtil {
     // The second byte in int would be the major version, 3rd byte minor version, and 4th byte 
     // patch version.
     public static int decodePhoenixVersion(long version) {
-        return (int) ((version << Byte.SIZE * 3) >>> Byte.SIZE * 4);
+        return (int) ((version << Byte.SIZE * 4) >>> Byte.SIZE * 5);
     }
     
     // TODO: generalize this to use two bytes to return a SQL error code instead
@@ -144,18 +145,43 @@ public class MetaDataUtil {
         int patch = version & 0xFF;
         return major + "." + minor + "." + patch;
     }
-
-    public static int encodePhoenixVersion() {
-        return VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION,
-                MetaDataProtocol.PHOENIX_PATCH_NUMBER);
+    
+    // Given the encoded integer representing the phoenix version in the encoded version value.
+    // The second byte in int would be the major version, 3rd byte minor version, and 4th byte
+    // patch version.
+    public static boolean decodeTableNamespaceMappingEnabled(long version) {
+        return ((int)((version << Byte.SIZE * 3) >>> Byte.SIZE * 7) & 0x1) != 0;
     }
 
-    public static long encodeHBaseAndPhoenixVersions(String hbaseVersion) {
-        return (((long) VersionUtil.encodeVersion(hbaseVersion)) << (Byte.SIZE * 5)) |
-                (((long) VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION,
-                        MetaDataProtocol.PHOENIX_PATCH_NUMBER)) << (Byte.SIZE * 1));
+    // The first 3 bytes of the long is used to encoding the HBase version as major.minor.patch.
+    // The next 4 bytes of the value is used to encode the Phoenix version as major.minor.patch.
+    /**
+     * Encode HBase and Phoenix version along with some server-side config information such as whether WAL codec is
+     * installed (necessary for non transactional, mutable secondar indexing), and whether systemNamespace mapping is enabled.
+     * 
+     * @param env
+     *            RegionCoprocessorEnvironment to access HBase version and Configuration.
+     * @return long value sent back during initialization of a cluster connection.
+     */
+    public static long encodeVersion(String hbaseVersionStr, Configuration config) {
+        long hbaseVersion = VersionUtil.encodeVersion(hbaseVersionStr);
+        long isTableNamespaceMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
+                new ReadOnlyProps(config.iterator())) ? 1 : 0;
+        long phoenixVersion = VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION,
+                MetaDataProtocol.PHOENIX_MINOR_VERSION, MetaDataProtocol.PHOENIX_PATCH_NUMBER);
+        long walCodec = IndexManagementUtil.isWALEditCodecSet(config) ? 0 : 1;
+        long version =
+        // Encode HBase major, minor, patch version
+        (hbaseVersion << (Byte.SIZE * 5))
+                // Encode if systemMappingEnabled are enabled on the server side
+                | (isTableNamespaceMappingEnabled << (Byte.SIZE * 4))
+                // Encode Phoenix major, minor, patch version
+                | (phoenixVersion << (Byte.SIZE * 1))
+                // Encode whether or not non transactional, mutable secondary indexing was configured properly.
+                | walCodec;
+        return version;
     }
-
+    
     public static void getTenantIdAndSchemaAndTableName(List<Mutation> tableMetadata, byte[][] rowKeyMetaData) {
         Mutation m = getTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index dbc65d4..cb1ef47 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -214,6 +214,10 @@ public class SchemaUtil {
         return name!=null && name.length() > 0 && name.charAt(0)=='"';
     }
     
+    private static boolean isExistingTableMappedToPhoenixName(String name) {
+        return name != null && name.length() > 0 && name.charAt(0) == '"' && name.indexOf("\"", 1) == name.length() - 1;
+    }
+
     public static <T> List<T> concat(List<T> l1, List<T> l2) {
         int size1 = l1.size();
         if (size1 == 0) {
@@ -597,8 +601,9 @@ public class SchemaUtil {
     }
 
     public static String getSchemaNameFromFullName(String tableName) {
-        if (tableName
-                .contains(":")) { return getSchemaNameFromFullName(tableName, QueryConstants.NAMESPACE_SEPARATOR); }
+        if (isExistingTableMappedToPhoenixName(tableName)) { return StringUtil.EMPTY_STRING; }
+        if (tableName.contains(QueryConstants.NAMESPACE_SEPARATOR)) { return getSchemaNameFromFullName(tableName,
+                QueryConstants.NAMESPACE_SEPARATOR); }
         return getSchemaNameFromFullName(tableName, QueryConstants.NAME_SEPARATOR);
     }
 
@@ -623,6 +628,7 @@ public class SchemaUtil {
         if (tableName == null) {
             return null;
         }
+        if (isExistingTableMappedToPhoenixName(Bytes.toString(tableName))) { return StringUtil.EMPTY_STRING; }
         int index = indexOf(tableName, QueryConstants.NAME_SEPARATOR_BYTE);
         if (index < 0) {
             index = indexOf(tableName, QueryConstants.NAMESPACE_SEPARATOR_BYTE);
@@ -635,6 +641,7 @@ public class SchemaUtil {
         if (tableName == null) {
             return null;
         }
+        if (isExistingTableMappedToPhoenixName(Bytes.toString(tableName))) { return normalizeIdentifier(Bytes.toString(tableName)); }
         int index = indexOf(tableName, QueryConstants.NAME_SEPARATOR_BYTE);
         if (index < 0) {
             index = indexOf(tableName, QueryConstants.NAMESPACE_SEPARATOR_BYTE);
@@ -644,7 +651,9 @@ public class SchemaUtil {
     }
 
     public static String getTableNameFromFullName(String tableName) {
-        if (tableName.contains(":")) { return getTableNameFromFullName(tableName, QueryConstants.NAMESPACE_SEPARATOR); }
+        if (isExistingTableMappedToPhoenixName(tableName)) { return normalizeIdentifier(tableName); }
+        if (tableName.contains(QueryConstants.NAMESPACE_SEPARATOR)) { return getTableNameFromFullName(tableName,
+                QueryConstants.NAMESPACE_SEPARATOR); }
         return getTableNameFromFullName(tableName, QueryConstants.NAME_SEPARATOR);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/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 c86d6b6..8d00b2b 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
@@ -1315,20 +1315,25 @@ public class UpgradeUtil {
                         ? "For system table " + QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE
                                 + " also needs to be enabled along with " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
                         : QueryServices.IS_NAMESPACE_MAPPING_ENABLED + " is not enabled"); }
+        boolean srcTableExists=admin.tableExists(srcTableName);
         // we need to move physical table in actual namespace for TABLE and Index
-        if (admin.tableExists(srcTableName) && (PTableType.TABLE.equals(pTableType)
+        if (srcTableExists && (PTableType.TABLE.equals(pTableType)
                 || PTableType.INDEX.equals(pTableType) || PTableType.SYSTEM.equals(pTableType))) {
-            String snapshotName = QueryConstants.UPGRADE_TABLE_SNAPSHOT_PREFIX + srcTableName;
-            logger.info("Disabling table " + srcTableName + " ..");
-            admin.disableTable(srcTableName);
-            logger.info(String.format("Taking snapshot %s of table %s..", snapshotName, srcTableName));
-            admin.snapshot(snapshotName, srcTableName);
-            logger.info(String.format("Restoring snapshot %s in destination table %s..", snapshotName, destTableName));
-            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(destTableName));
-            logger.info(String.format("deleting old table %s..", srcTableName));
-            admin.deleteTable(srcTableName);
-            logger.info(String.format("deleting snapshot %s..", snapshotName));
-            admin.deleteSnapshot(snapshotName);
+            boolean destTableExists=admin.tableExists(destTableName);
+            if (!destTableExists) {
+                String snapshotName = QueryConstants.UPGRADE_TABLE_SNAPSHOT_PREFIX + srcTableName;
+                logger.info("Disabling table " + srcTableName + " ..");
+                admin.disableTable(srcTableName);
+                logger.info(String.format("Taking snapshot %s of table %s..", snapshotName, srcTableName));
+                admin.snapshot(snapshotName, srcTableName);
+                logger.info(
+                        String.format("Restoring snapshot %s in destination table %s..", snapshotName, destTableName));
+                admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(destTableName));
+                logger.info(String.format("deleting old table %s..", srcTableName));
+                admin.deleteTable(srcTableName);
+                logger.info(String.format("deleting snapshot %s..", snapshotName));
+                admin.deleteSnapshot(snapshotName);
+            }
         }
         // Update flag to represent table is mapped to namespace
         logger.info(String.format("Updating meta information of phoenix table '%s' to map to namespace..", phoenixTableName));
@@ -1358,6 +1363,9 @@ public class UpgradeUtil {
                 "May not specify the TENANT_ID_ATTRIB property when upgrading"); }
         if (conn.getSchema() != null) { throw new IllegalArgumentException(
                 "Schema should not be set for connection!!"); }
+        if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
+                readOnlyProps)) { throw new IllegalArgumentException(
+                        QueryServices.IS_NAMESPACE_MAPPING_ENABLED + " is not enabled!!"); }
         try (HBaseAdmin admin = conn.getQueryServices().getAdmin();
                 HTableInterface metatable = conn.getQueryServices()
                         .getTable(SchemaUtil
@@ -1365,22 +1373,24 @@ public class UpgradeUtil {
                                 .getName());) {
             String tableName = SchemaUtil.normalizeIdentifier(srcTable);
             String schemaName = SchemaUtil.getSchemaNameFromFullName(tableName);
-
-            // Upgrade is not required if schemaName is not present.
-            if (schemaName.equals("")) { throw new IllegalArgumentException("Table doesn't have schema name"); }
-
             // Confirm table is not already upgraded
             PTable table = PhoenixRuntime.getTable(conn, tableName);
+            // Upgrade is not required if schemaName is not present.
+            if (schemaName.equals("") && !PTableType.VIEW
+                    .equals(table.getType())) { throw new IllegalArgumentException("Table doesn't have schema name"); }
+
             if (table.isNamespaceMapped()) { throw new IllegalArgumentException("Table is already upgraded"); }
-            logger.info(String.format("Creating schema %s..", schemaName));
-            conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
+            if (!schemaName.equals("")) {
+                logger.info(String.format("Creating schema %s..", schemaName));
+                conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
+            }
             String newPhysicalTablename = SchemaUtil.normalizeIdentifier(SchemaUtil
                     .getPhysicalTableName(table.getPhysicalName().getString(), readOnlyProps).getNameAsString());
             logger.info(String.format("Upgrading %s %s..", table.getType(), tableName));
             // Upgrade the data or main table
             mapTableToNamespace(admin, metatable, tableName, newPhysicalTablename, readOnlyProps,
                     PhoenixRuntime.getCurrentScn(readOnlyProps), tableName, table.getType());
-
+            conn.close();
             // clear the cache and get new table
             conn.getQueryServices().clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY, table.getSchemaName().getBytes(),
                     table.getTableName().getBytes(), PhoenixRuntime.getCurrentScn(readOnlyProps));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20ce95b0/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
index 1e06379..0443b77 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
@@ -21,15 +21,19 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.VersionInfo;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
 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.util.VersionUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.HBaseFactoryProvider;
+import org.apache.phoenix.query.QueryServices;
 import org.junit.Test;
 
 
@@ -103,5 +107,37 @@ public class MetaDataUtilTest {
   private static ImmutableBytesPtr wrap(byte[] bytes) {
     return new ImmutableBytesPtr(bytes);
   }
+
+    @Test
+    public void testEncodeDecode() {
+        String hbaseVersionStr = "0.98.14";
+        Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+        config.setBoolean(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, false);
+        config.setBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, false);
+
+        long version = MetaDataUtil.encodeVersion(hbaseVersionStr, config);
+        int hbaseVersion = MetaDataUtil.decodeHBaseVersion(version);
+        int expectedHBaseVersion = VersionUtil.encodeVersion(0, 98, 14);
+        assertEquals(expectedHBaseVersion, hbaseVersion);
+        boolean isTableNamespaceMappingEnabled = MetaDataUtil.decodeTableNamespaceMappingEnabled(version);
+        assertFalse(isTableNamespaceMappingEnabled);
+        int phoenixVersion = MetaDataUtil.decodePhoenixVersion(version);
+        int expectedPhoenixVersion = VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION,
+                MetaDataProtocol.PHOENIX_MINOR_VERSION, MetaDataProtocol.PHOENIX_PATCH_NUMBER);
+        assertEquals(expectedPhoenixVersion, phoenixVersion);
+
+        config.setBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, true);
+
+        version = MetaDataUtil.encodeVersion(hbaseVersionStr, config);
+        hbaseVersion = MetaDataUtil.decodeHBaseVersion(version);
+        expectedHBaseVersion = VersionUtil.encodeVersion(0, 98, 14);
+        assertEquals(expectedHBaseVersion, hbaseVersion);
+        isTableNamespaceMappingEnabled = MetaDataUtil.decodeTableNamespaceMappingEnabled(version);
+        assertTrue(isTableNamespaceMappingEnabled);
+        phoenixVersion = MetaDataUtil.decodePhoenixVersion(version);
+        expectedPhoenixVersion = VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION,
+                MetaDataProtocol.PHOENIX_MINOR_VERSION, MetaDataProtocol.PHOENIX_PATCH_NUMBER);
+        assertEquals(expectedPhoenixVersion, phoenixVersion);
+    }
 }
 


[3/3] phoenix git commit: PHOENIX-2887 Uberjar application fail with 'IllegalAccessError: class com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass com.google.protobuf.LiteralByteString'

Posted by an...@apache.org.
PHOENIX-2887 Uberjar application fail with 'IllegalAccessError: class com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass com.google.protobuf.LiteralByteString'


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

Branch: refs/heads/master
Commit: 5d641ad34207b8e87b1f460a168664387f143286
Parents: 20ce95b
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon May 16 12:17:05 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon May 16 12:17:05 2016 +0530

----------------------------------------------------------------------
 dev/test-patch.properties                       |  1 +
 dev/test-patch.sh                               | 32 ++++++++++++++++++++
 .../apache/phoenix/cache/ServerCacheClient.java |  3 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  8 ++---
 4 files changed, 37 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5d641ad3/dev/test-patch.properties
----------------------------------------------------------------------
diff --git a/dev/test-patch.properties b/dev/test-patch.properties
index 53f2ad4..8d1e2ed 100644
--- a/dev/test-patch.properties
+++ b/dev/test-patch.properties
@@ -33,3 +33,4 @@ BRANCH_NAMES="4.4-HBase-0.98 4.4-HBase-1.0 4.4-HBase-1.1 4.x-HBase-0.98 4.x-HBas
 # All supported Hadoop versions that we want to test the compilation with
 HADOOP2_VERSIONS="2.4.1 2.5.2 2.6.0"
 HADOOP3_VERSIONS="3.0.0-SNAPSHOT"
+INCOMPATIBLE_CLASSES="com.google.protobuf.HBaseZeroCopyByteString"

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5d641ad3/dev/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev/test-patch.sh b/dev/test-patch.sh
index 5ee1607..4454e13 100755
--- a/dev/test-patch.sh
+++ b/dev/test-patch.sh
@@ -520,6 +520,36 @@ checkAntiPatterns () {
 }
 
 ###############################################################################
+### Check against use of known incompatible classes
+checkUseOfIncompatibleClasses () {
+  echo ""
+  echo ""
+  echo "======================================================================"
+  echo "======================================================================"
+  echo "    Checking against use of known incompatible classes."
+  echo "======================================================================"
+  echo "======================================================================"
+  echo ""
+  echo ""
+  for INCOMPATIBLE_CLASS in $INCOMPATIBLE_CLASSES ; do
+  warning=`$GREP "+import $INCOMPATIBLE_CLASS" $PATCH_DIR/patch`
+  if [[ $warning != "" ]]; then
+        warnings="$warnings
+                $INCOMPATIBLE_CLASS"
+        (( count = count + 1 ))
+  fi
+  done
+  if [[ $warnings != "" ]]; then
+    JIRA_COMMENT="$JIRA_COMMENT
+
+                    {color:red}-1 Known Incompatible class{color}.  The patch appears to have $count incompatible classes:
+             $warnings."
+          return 1
+  fi
+  return 0
+}
+
+###############################################################################
 ### Check that there are no incorrect annotations
 checkInterfaceAudience () {
   echo ""
@@ -1050,6 +1080,8 @@ fi
 
 checkAntiPatterns
 (( RESULT = RESULT + $? ))
+checkUseOfIncompatibleClasses
+(( RESULT = RESULT + $? ))
 # checkBuildWithHadoopVersions
 # (( RESULT = RESULT + $? ))
 checkJavacWarnings

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5d641ad3/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
index 424482a..aea15c2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
@@ -71,7 +71,6 @@ import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.collect.ImmutableSet;
-import com.google.protobuf.HBaseZeroCopyByteString;
 
 /**
  * 
@@ -214,7 +213,7 @@ public class ServerCacheClient {
                                                     ServerCacheFactoryProtos.ServerCacheFactory.Builder svrCacheFactoryBuider = ServerCacheFactoryProtos.ServerCacheFactory.newBuilder();
                                                     svrCacheFactoryBuider.setClassName(cacheFactory.getClass().getName());
                                                     builder.setCacheFactory(svrCacheFactoryBuider.build());
-                                                    builder.setTxState(HBaseZeroCopyByteString.wrap(txState));
+                                                    builder.setTxState(ByteStringer.wrap(txState));
                                                     instance.addServerCache(controller, builder.build(), rpcCallback);
                                                     if(controller.getFailedOn() != null) {
                                                         throw controller.getFailedOn();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5d641ad3/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 70414d5..bd23ab8 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
@@ -66,8 +66,6 @@ import org.apache.phoenix.util.SizedUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 
-import co.cask.tephra.TxConstants;
-
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
@@ -77,9 +75,10 @@ import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.protobuf.HBaseZeroCopyByteString;
 import com.sun.istack.NotNull;
 
+import co.cask.tephra.TxConstants;
+
 /**
  *
  * Base class for PTable implementors.  Provides abstraction for
@@ -1069,8 +1068,7 @@ public class PTableImpl implements PTable {
           int maxLength = pGuidePosts.getMaxLength();
           int guidePostsCount = pGuidePosts.getEncodedGuidePostsCount();
             GuidePostsInfo info = new GuidePostsInfo(pGuidePosts.getByteCountsList(),
-                    new ImmutableBytesWritable(
-                            HBaseZeroCopyByteString.zeroCopyGetBytes(pGuidePosts.getEncodedGuidePosts())),
+                    new ImmutableBytesWritable(pGuidePosts.getEncodedGuidePosts().toByteArray()),
                     pGuidePosts.getRowCountsList(), maxLength, guidePostsCount);
           tableGuidePosts.put(pTableStatsProto.getKey().toByteArray(), info);
       }