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

[01/50] [abbrv] phoenix git commit: Revert "PHOENIX-4790 Simplify check for client side delete"

Repository: phoenix
Updated Branches:
  refs/heads/omid2 119212c3f -> 24178207a


Revert "PHOENIX-4790 Simplify check for client side delete"

This reverts commit 6918b0dfba81f1da21b51965a91269115da67acc.


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

Branch: refs/heads/omid2
Commit: 4cab4c270e91b48b4ebd19487986cfd864e02476
Parents: e95fb21
Author: James Taylor <ja...@apache.org>
Authored: Thu Jul 12 19:57:14 2018 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu Jul 12 19:57:14 2018 -0700

----------------------------------------------------------------------
 .../apache/phoenix/compile/DeleteCompiler.java  | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4cab4c27/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 78b2db9..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
@@ -46,6 +46,7 @@ import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
+import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.hbase.index.ValueGetter;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -480,7 +481,6 @@ public class DeleteCompiler {
             projectedColumns.add(column);
             aliasedNodes.add(FACTORY.aliasedNode(null, FACTORY.column(null, '"' + column.getName().getString() + '"', null)));
         }
-        boolean noQueryReqd = true;
         // Project all non PK indexed columns so that we can do the proper index maintenance
         for (PTable index : table.getIndexes()) {
             IndexMaintainer maintainer = index.getIndexMaintainer(table, connection);
@@ -492,8 +492,6 @@ public class DeleteCompiler {
                     boolean hasNoColumnFamilies = table.getColumnFamilies().isEmpty();
                     PColumn column = hasNoColumnFamilies ? table.getColumnForColumnName(columnName) : table.getColumnFamily(familyName).getPColumnForColumnName(columnName);
                     if(!projectedColumns.contains(column)) {
-                        // We must run a query if any index contains a non pk column
-                        noQueryReqd = false;
                         projectedColumns.add(column);
                         aliasedNodes.add(FACTORY.aliasedNode(null, FACTORY.column(hasNoColumnFamilies ? null : TableName.create(null, familyName), '"' + columnName + '"', null)));
                     }
@@ -513,7 +511,7 @@ public class DeleteCompiler {
             select = StatementNormalizer.normalize(transformedSelect, resolverToBe);
         }
         final boolean hasPreOrPostProcessing = hasPreProcessing || hasPostProcessing;
-        noQueryReqd &= !hasPreOrPostProcessing;
+        boolean noQueryReqd = !hasPreOrPostProcessing;
         // No limit and no sub queries, joins, etc in where clause
         // Can't run on same server for transactional data, as we need the row keys for the data
         // that is being upserted for conflict detection purposes.
@@ -552,8 +550,24 @@ public class DeleteCompiler {
         }
         
         runOnServer &= queryPlans.get(0).getTableRef().getTable().getType() != PTableType.INDEX;
-
+        
+        // We need to have all indexed columns available in all immutable indexes in order
+        // to generate the delete markers from the query. We also cannot have any filters
+        // except for our SkipScanFilter for point lookups.
+        // A simple check of the non existence of a where clause in the parse node is not sufficient, as the where clause
+        // may have been optimized out. Instead, we check that there's a single SkipScanFilter
+        // If we can generate a plan for every index, that means all the required columns are available in every index,
+        // hence we can drive the delete from any of the plans.
         noQueryReqd &= queryPlans.size() == 1 + clientSideIndexes.size();
+        int queryPlanIndex = 0;
+        while (noQueryReqd && queryPlanIndex < queryPlans.size()) {
+            QueryPlan plan = queryPlans.get(queryPlanIndex++);
+            StatementContext context = plan.getContext();
+            noQueryReqd &= (!context.getScan().hasFilter()
+                    || context.getScan().getFilter() instanceof SkipScanFilter)
+                && context.getScanRanges().isPointLookup();
+        }
+
         final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
         final int maxSizeBytes = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
  


[41/50] [abbrv] phoenix git commit: PHOENIX-4870 LoggingPhoenixConnection should log metrics when AutoCommit is set to True.

Posted by ja...@apache.org.
PHOENIX-4870 LoggingPhoenixConnection should log metrics when AutoCommit is set to True.


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

Branch: refs/heads/omid2
Commit: d105fbb992b0c0221691cb64260907786954842a
Parents: 7e1eabf
Author: s.kadam <s....@gus.com>
Authored: Wed Sep 5 17:00:03 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Thu Sep 6 23:51:30 2018 -0700

----------------------------------------------------------------------
 .../monitoring/PhoenixLoggingMetricsIT.java     | 61 +++++++++++++++++++-
 .../phoenix/jdbc/LoggingPhoenixConnection.java  | 37 +++++++-----
 .../jdbc/LoggingPhoenixPreparedStatement.java   | 25 +++++++-
 .../phoenix/jdbc/LoggingPhoenixStatement.java   | 28 +++++++--
 4 files changed, 125 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d105fbb9/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
index 5d5524c..483d341 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
@@ -102,7 +102,8 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     public void testPhoenixMetricsLoggedOnCommit() throws Exception {
         // run SELECT to verify read metrics are logged
         String query = "SELECT * FROM " + tableName1;
-        verifyQueryLevelMetricsLogging(query);
+        ResultSet rs = upsertRows(query);
+        verifyQueryLevelMetricsLogging(query, rs);
 
         // run UPSERT SELECT to verify mutation metrics are logged
         String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
@@ -140,7 +141,9 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     public void testPhoenixMetricsLoggedOnClose() throws Exception {
         // run SELECT to verify read metrics are logged
         String query = "SELECT * FROM " + tableName1;
-        verifyQueryLevelMetricsLogging(query);
+
+        ResultSet rs = upsertRows(query);
+        verifyQueryLevelMetricsLogging(query, rs);
 
         // run UPSERT SELECT to verify mutation metrics are logged
         String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
@@ -164,13 +167,61 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
                 mutationReadMetricsMap.size() == 0);
     }
 
+    /**
+     * This test is added to verify if metrics are being logged in case
+     * auto commit is set to true.
+     */
+    @Test
+    public void testPhoenixMetricsLoggedOnAutoCommitTrue() throws Exception {
+        loggedConn.setAutoCommit(true);
+
+        String query = "SELECT * FROM " + tableName1;
+        ResultSet rs = upsertRows(query);
+        verifyQueryLevelMetricsLogging(query, rs);
+
+        // run UPSERT SELECT to verify mutation metrics are logged
+        String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
+        loggedConn.createStatement().executeUpdate(upsertSelect);
+
+        assertTrue("Mutation write metrics are not logged for " + tableName2,
+                mutationWriteMetricsMap.get(tableName2).size()  > 0);
+        assertTrue("Mutation read metrics are not found for " + tableName1,
+                mutationReadMetricsMap.get(tableName1).size() > 0);
+
+        clearAllTestMetricMaps();
+
+        loggedConn.createStatement().execute(query);
+        assertTrue("Read metrics found for " + tableName1,
+                mutationReadMetricsMap.size() == 0);
+        loggedConn.createStatement().execute(upsertSelect);
+
+        assertTrue("Mutation write metrics are not logged for " + tableName2
+                + " in createStatement",mutationWriteMetricsMap.get(tableName2).size()  > 0);
+        assertTrue("Mutation read metrics are not found for " + tableName1
+                + " in createStatement",mutationReadMetricsMap.get(tableName1).size() > 0);
+
+        clearAllTestMetricMaps();
+
+        loggedConn.prepareStatement(query).executeQuery();
+        assertTrue("Read metrics found for " + tableName1,
+                mutationReadMetricsMap.size() == 0);
+
+        loggedConn.prepareStatement(upsertSelect).executeUpdate();
+        assertTrue("Mutation write metrics are not logged for " + tableName2
+                + " in prepareStatement",mutationWriteMetricsMap.get(tableName2).size()  > 0);
+        assertTrue("Mutation read metrics are not found for " + tableName1
+                + " in prepareStatement",mutationReadMetricsMap.get(tableName1).size() > 0);
+
+
+    }
+
     private ResultSet executeAndGetResultSet(String query) throws Exception {
         Statement stmt = loggedConn.createStatement();
         stmt.execute(query);
         return stmt.getResultSet();
     }
 
-    private void verifyQueryLevelMetricsLogging(String query) throws SQLException {
+    private ResultSet upsertRows(String query) throws SQLException {
         Statement stmt = loggedConn.createStatement();
         ResultSet rs = stmt.executeQuery(query);
         assertTrue(rs instanceof LoggingPhoenixResultSet);
@@ -180,6 +231,10 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
         }
         rs.close();
         assertTrue(rowsRetrievedCounter == NUM_ROWS);
+        return rs;
+    }
+
+    private void verifyQueryLevelMetricsLogging(String query , ResultSet rs) throws SQLException {
         assertTrue("Read metrics for not found for " + tableName1,
                 requestReadMetricsMap.get(tableName1).size() > 0);
         assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d105fbb9/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
index 37917e2..af0f803 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
@@ -40,14 +40,16 @@ public class LoggingPhoenixConnection extends DelegateConnection {
 
     @Override
     public Statement createStatement() throws SQLException {
-        return new LoggingPhoenixStatement(super.createStatement(), phoenixMetricsLog);
+        return new LoggingPhoenixStatement(super.createStatement(), phoenixMetricsLog,
+                this);
     }
 
     @Override
     public Statement createStatement(int resultSetType, int resultSetConcurrency)
             throws SQLException {
         return new LoggingPhoenixStatement(
-                super.createStatement(resultSetType, resultSetConcurrency), phoenixMetricsLog);
+                super.createStatement(resultSetType, resultSetConcurrency), phoenixMetricsLog,
+                this);
     }
 
     @Override
@@ -55,13 +57,13 @@ public class LoggingPhoenixConnection extends DelegateConnection {
             int resultSetHoldability) throws SQLException {
         return new LoggingPhoenixStatement(
                 super.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability),
-                phoenixMetricsLog);
+                phoenixMetricsLog, this);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql) throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql),
-                phoenixMetricsLog, sql);
+                phoenixMetricsLog, sql, this);
     }
 
     @Override
@@ -69,52 +71,57 @@ public class LoggingPhoenixConnection extends DelegateConnection {
             int resultSetConcurrency) throws SQLException {
         return new LoggingPhoenixPreparedStatement(
                 super.prepareStatement(sql, resultSetType, resultSetConcurrency),
-                phoenixMetricsLog, sql);
+                phoenixMetricsLog, sql, this);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, int resultSetType,
             int resultSetConcurrency, int resultSetHoldability) throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, resultSetType,
-            resultSetConcurrency, resultSetHoldability), phoenixMetricsLog, sql);
+            resultSetConcurrency, resultSetHoldability), phoenixMetricsLog, sql, this);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys)
             throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, autoGeneratedKeys),
-                phoenixMetricsLog, sql);
+                phoenixMetricsLog, sql, this);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, columnIndexes),
-                phoenixMetricsLog, sql);
+                phoenixMetricsLog, sql, this);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, String[] columnNames)
             throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, columnNames),
-                phoenixMetricsLog, sql);
+                phoenixMetricsLog, sql, this);
     }
 
     @Override
     public void commit() throws SQLException {
         super.commit();
-        phoenixMetricsLog.logWriteMetricsfoForMutationsSinceLastReset(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
-        phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
-        PhoenixRuntime.resetMetrics(conn);
+        loggingMetricsHelper();
     }
 
     @Override
     public void close() throws SQLException {
         try {
-            phoenixMetricsLog.logWriteMetricsfoForMutationsSinceLastReset(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
-            phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
-            PhoenixRuntime.resetMetrics(conn);
+            loggingMetricsHelper();
         } finally {
             super.close();
         }
     }
+
+    public void loggingMetricsHelper() throws SQLException {
+
+        phoenixMetricsLog.logWriteMetricsfoForMutationsSinceLastReset(
+                PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
+        phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(
+                PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
+        PhoenixRuntime.resetMetrics(conn);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d105fbb9/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
index dbeea0d..12edde9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
@@ -21,16 +21,21 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
+import java.sql.Connection;
+
 
 public class LoggingPhoenixPreparedStatement extends DelegatePreparedStatement {
     
     private PhoenixMetricsLog phoenixMetricsLog;
     private String sql;
+    private Connection conn;
     
-    public LoggingPhoenixPreparedStatement(PreparedStatement stmt, PhoenixMetricsLog phoenixMetricsLog, String sql) {
+    public LoggingPhoenixPreparedStatement(PreparedStatement stmt,
+                                           PhoenixMetricsLog phoenixMetricsLog, String sql, Connection conn) {
         super(stmt);
         this.phoenixMetricsLog = phoenixMetricsLog;
         this.sql = sql;
+        this.conn = conn;
     }
     
     @Override
@@ -40,7 +45,9 @@ public class LoggingPhoenixPreparedStatement extends DelegatePreparedStatement {
 
     @Override
     public ResultSet executeQuery() throws SQLException {
-        return new LoggingPhoenixResultSet(super.executeQuery(), phoenixMetricsLog, sql);
+        ResultSet rs = new LoggingPhoenixResultSet(super.executeQuery(), phoenixMetricsLog, sql);
+        this.loggingAutoCommitHelper();
+        return rs;
     }
 
     @Override
@@ -50,10 +57,22 @@ public class LoggingPhoenixPreparedStatement extends DelegatePreparedStatement {
         return (resultSet == null) ? null : new LoggingPhoenixResultSet(resultSet,
                 phoenixMetricsLog, sql);
     }
+
+    @Override
+    public int executeUpdate() throws SQLException {
+        int res = super.executeUpdate();
+        this.loggingAutoCommitHelper();
+        return res;
+    }
     
     @Override
     public ResultSet getGeneratedKeys() throws SQLException {
         return new LoggingPhoenixResultSet(super.getGeneratedKeys(), phoenixMetricsLog, sql);
     }
-    
+
+    private void loggingAutoCommitHelper() throws SQLException {
+        if(conn.getAutoCommit() && (conn instanceof LoggingPhoenixConnection)) {
+            ((LoggingPhoenixConnection)conn).loggingMetricsHelper();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d105fbb9/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
index de33893..d31f521 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.jdbc;
 
+
+import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -25,28 +27,38 @@ public class LoggingPhoenixStatement extends DelegateStatement {
 
     private PhoenixMetricsLog phoenixMetricsLog;
     private String sql;
-    
-    public LoggingPhoenixStatement(Statement stmt, PhoenixMetricsLog phoenixMetricsLog) {
+    private Connection conn;
+
+    public LoggingPhoenixStatement(Statement stmt, PhoenixMetricsLog phoenixMetricsLog, Connection conn) {
         super(stmt);
         this.phoenixMetricsLog = phoenixMetricsLog;
+        this.conn = conn;
     }
 
     @Override
     public boolean execute(String sql) throws SQLException {
+        boolean result;
         this.sql = sql;
-        return super.execute(sql);
+        result = super.execute(sql);
+        this.loggingAutoCommitHelper();
+        return result;
     }
 
     @Override
     public ResultSet executeQuery(String sql) throws SQLException {
         this.sql = sql;
-        return new LoggingPhoenixResultSet(super.executeQuery(sql), phoenixMetricsLog, this.sql);
+        ResultSet rs = new LoggingPhoenixResultSet(super.executeQuery(sql), phoenixMetricsLog, this.sql);
+        this.loggingAutoCommitHelper();
+        return rs;
     }
 
     @Override
     public int executeUpdate(String sql) throws SQLException {
+        int result;
         this.sql = sql;
-        return super.executeUpdate(sql);
+        result = super.executeUpdate(sql);
+        this.loggingAutoCommitHelper();
+        return result;
     }
 
     @Override
@@ -62,4 +74,10 @@ public class LoggingPhoenixStatement extends DelegateStatement {
         return new LoggingPhoenixResultSet(super.getGeneratedKeys(), phoenixMetricsLog, this.sql);
     }
 
+    private void loggingAutoCommitHelper() throws SQLException {
+        if(conn.getAutoCommit() && (conn instanceof LoggingPhoenixConnection)) {
+            ((LoggingPhoenixConnection)conn).loggingMetricsHelper();
+        }
+    }
+
 }


[31/50] [abbrv] phoenix git commit: PHOENIX-4755 Provide an option to plugin custom avatica server config in PQS

Posted by ja...@apache.org.
PHOENIX-4755 Provide an option to plugin custom avatica server config in PQS


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

Branch: refs/heads/omid2
Commit: 0639a7426558cf298288e05c193eea438c2f07e5
Parents: 7a2edb8
Author: Karan Mehta <ka...@gmail.com>
Authored: Mon Aug 20 16:52:22 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Tue Aug 21 10:15:49 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/query/QueryServices.java |   3 +-
 .../phoenix/query/QueryServicesOptions.java     |   1 +
 .../phoenix/end2end/ServerCustomizersIT.java    |   4 +-
 .../AvaticaServerConfigurationFactory.java      |  20 +++
 .../phoenix/queryserver/server/QueryServer.java | 167 ++++++++++++-------
 .../server/ServerCustomizersFactory.java        |   7 +-
 .../CustomAvaticaServerConfigurationTest.java   |  20 +++
 .../server/QueryServerConfigurationTest.java    |  26 ++-
 .../server/ServerCustomizersTest.java           |  13 +-
 9 files changed, 194 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/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 d290174..d681a13 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
@@ -250,8 +250,9 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_KERBEROS_ALLOWED_REALMS = "phoenix.queryserver.kerberos.allowed.realms";
     public static final String QUERY_SERVER_SPNEGO_AUTH_DISABLED_ATTRIB = "phoenix.queryserver.spnego.auth.disabled";
     public static final String QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR_ATTRIB = "phoenix.queryserver.withRemoteUserExtractor";
-    public static final String QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "phoenix.queryserver.remoteUserExtractor.param";
     public static final String QUERY_SERVER_CUSTOMIZERS_ENABLED = "phoenix.queryserver.customizers.enabled";
+    public static final String QUERY_SERVER_CUSTOM_AUTH_ENABLED = "phoenix.queryserver.custom.auth.enabled";
+    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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/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 c68e793..76e79fa 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
@@ -299,6 +299,7 @@ public class QueryServicesOptions {
     public static final int DEFAULT_QUERY_SERVER_UGI_CACHE_CONCURRENCY = 10;
     public static final boolean DEFAULT_QUERY_SERVER_SPNEGO_AUTH_DISABLED = false;
     public static final boolean DEFAULT_QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR = false;
+    public static final boolean DEFAULT_QUERY_SERVER_CUSTOM_AUTH_ENABLED = false;
     public static final String DEFAULT_QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "doAs";
     public static final boolean DEFAULT_QUERY_SERVER_DISABLE_KERBEROS_LOGIN = false;
     public static final boolean DEFAULT_QUERY_SERVER_CUSTOMIZERS_ENABLED = false;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
index d990adb..db08908 100644
--- a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
+++ b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.ServerCustomizer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.query.QueryServices;
@@ -66,7 +67,8 @@ public class ServerCustomizersIT extends BaseHBaseManagedTimeIT {
         InstanceResolver.clearSingletons();
         InstanceResolver.getSingleton(ServerCustomizersFactory.class, new ServerCustomizersFactory() {
             @Override
-            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf,
+                                                                          AvaticaServerConfiguration avaticaServerConfiguration) {
                 return Collections.<ServerCustomizer<Server>>singletonList(new TestServerCustomizer());
             }
         });

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
new file mode 100644
index 0000000..87a72ea
--- /dev/null
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
@@ -0,0 +1,20 @@
+package org.apache.phoenix.queryserver.server;
+
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+
+public interface AvaticaServerConfigurationFactory {
+
+    AvaticaServerConfiguration getAvaticaServerConfiguration(Configuration conf, UserGroupInformation ugi);
+
+    class AvaticaServerConfigurationFactoryImpl implements AvaticaServerConfigurationFactory {
+
+        @Override
+        public AvaticaServerConfiguration getAvaticaServerConfiguration(Configuration conf, UserGroupInformation ugi) {
+            return null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/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 47466c8..4766394 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
@@ -27,6 +27,7 @@ import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.remote.Driver;
 import org.apache.calcite.avatica.remote.LocalService;
 import org.apache.calcite.avatica.remote.Service;
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.DoAsRemoteUserCallback;
 import org.apache.calcite.avatica.server.HttpServer;
 import org.apache.calcite.avatica.server.RemoteUserExtractor;
@@ -76,6 +77,8 @@ import java.util.concurrent.TimeUnit;
 
 import javax.servlet.http.HttpServletRequest;
 
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_QUERY_SERVER_CUSTOM_AUTH_ENABLED;
+
 /**
  * A query server for Phoenix over Calcite's Avatica.
  */
@@ -219,27 +222,38 @@ public final class QueryServer extends Configured implements Tool, Runnable {
         LOG.info(" Kerberos is off and hostname is : "+hostname);
       }
 
-      Class<? extends PhoenixMetaFactory> factoryClass = getConf().getClass(
-          QueryServices.QUERY_SERVER_META_FACTORY_ATTRIB, PhoenixMetaFactoryImpl.class,
-          PhoenixMetaFactory.class);
       int port = getConf().getInt(QueryServices.QUERY_SERVER_HTTP_PORT_ATTRIB,
           QueryServicesOptions.DEFAULT_QUERY_SERVER_HTTP_PORT);
       LOG.debug("Listening on port " + port);
-      PhoenixMetaFactory factory =
-          factoryClass.getDeclaredConstructor(Configuration.class).newInstance(getConf());
-      Meta meta = factory.create(Arrays.asList(args));
-      Service service = new LocalService(meta);
+
+      // Update proxyuser configuration for impersonation
+      ProxyUsers.refreshSuperUserGroupsConfiguration(getConf());
 
       // Start building the Avatica HttpServer
-      final HttpServer.Builder<Server> builder = HttpServer.Builder.<Server> newBuilder()
-          .withPort(port).withHandler(service, getSerialization(getConf()));
+      final HttpServer.Builder<Server>
+              builder =
+              HttpServer.Builder.<Server>newBuilder().withPort(port);
+
+      UserGroupInformation ugi = getUserGroupInformation();
 
-      // Enable client auth when using Kerberos auth for HBase
-      if (isKerberos) {
-        configureClientAuthentication(builder, disableSpnego);
+      AvaticaServerConfiguration avaticaServerConfiguration = null;
+
+      // RemoteUserCallbacks and RemoteUserExtractor are part of AvaticaServerConfiguration
+      // Hence they should be customizable when using QUERY_SERVER_CUSTOM_AUTH_ENABLED
+      // Handlers should be customized via ServerCustomizers
+      if (getConf().getBoolean(QueryServices.QUERY_SERVER_CUSTOM_AUTH_ENABLED,
+              DEFAULT_QUERY_SERVER_CUSTOM_AUTH_ENABLED)) {
+        avaticaServerConfiguration = enableCustomAuth(builder, getConf(), ugi);
+      } else {
+        if (isKerberos) {
+          // Enable client auth when using Kerberos auth for HBase
+          configureClientAuthentication(builder, disableSpnego, ugi);
+        }
+        setRemoteUserExtractorIfNecessary(builder, getConf());
+        setHandler(args, builder);
       }
-      setRemoteUserExtractorIfNecessary(builder, getConf());
-      enableServerCustomizersIfNecessary(builder, getConf());
+
+      enableServerCustomizersIfNecessary(builder, getConf(), avaticaServerConfiguration);
 
       // Build and start the HttpServer
       server = builder.build();
@@ -262,48 +276,71 @@ public final class QueryServer extends Configured implements Tool, Runnable {
   }
 
   @VisibleForTesting
-  void configureClientAuthentication(final HttpServer.Builder builder, boolean disableSpnego) throws IOException {
+  void configureClientAuthentication(final HttpServer.Builder builder, boolean disableSpnego, UserGroupInformation ugi) throws IOException {
+
+    // Enable SPNEGO for client authentication unless it's explicitly disabled
+    if (!disableSpnego) {
+      configureSpnegoAuthentication(builder, ugi);
+    }
+    configureCallBack(builder, ugi);
+  }
+
+  @VisibleForTesting
+  void configureSpnegoAuthentication(HttpServer.Builder builder, UserGroupInformation ugi) {
+    String keytabPath = getConf().get(QueryServices.QUERY_SERVER_KEYTAB_FILENAME_ATTRIB);
+    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);
+    }
+
+    String realmsString = getConf().get(QueryServices.QUERY_SERVER_KERBEROS_ALLOWED_REALMS, null);
+    String[] additionalAllowedRealms = null;
+    if (null != realmsString) {
+      additionalAllowedRealms = StringUtils.split(realmsString, ',');
+    }
+    if (null != httpKeytabPath && null != httpPrincipal) {
+      builder.withSpnego(httpPrincipal, additionalAllowedRealms).withAutomaticLogin(httpKeytab);
+    } else {
+      builder.withSpnego(ugi.getUserName(), additionalAllowedRealms)
+              .withAutomaticLogin(keytab);
+    }
+  }
+
+  @VisibleForTesting
+  UserGroupInformation getUserGroupInformation() throws IOException {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     LOG.debug("Current user is " + ugi);
     if (!ugi.hasKerberosCredentials()) {
       ugi = UserGroupInformation.getLoginUser();
       LOG.debug("Current user does not have Kerberos credentials, using instead " + ugi);
     }
+    return ugi;
+  }
 
-    // Make sure the proxyuser configuration is up to date
-    ProxyUsers.refreshSuperUserGroupsConfiguration(getConf());
-
-    // Always enable impersonation for the proxy user (through standard Hadoop configuration means)
+  @VisibleForTesting
+  void configureCallBack(HttpServer.Builder<Server> builder, UserGroupInformation ugi) {
     builder.withImpersonation(new PhoenixDoAsCallback(ugi, getConf()));
+  }
 
-    // Enable SPNEGO for client authentication unless it's explicitly disabled
-    if (!disableSpnego) {
-      String keytabPath = getConf().get(QueryServices.QUERY_SERVER_KEYTAB_FILENAME_ATTRIB);
-      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);
-
-      String realmsString = getConf().get(QueryServices.QUERY_SERVER_KERBEROS_ALLOWED_REALMS, null);
-      String[] additionalAllowedRealms = null;
-      if (null != realmsString) {
-        additionalAllowedRealms = StringUtils.split(realmsString, ',');
-      }
-      if ((null != httpKeytabPath) && (null != httpPrincipal)) {
-        builder.withSpnego(httpPrincipal, additionalAllowedRealms).withAutomaticLogin(httpKeytab);
-      } else {
-        builder.withSpnego(ugi.getUserName(), additionalAllowedRealms)
-            .withAutomaticLogin(keytab);
-      }
-    }
+  private void setHandler(String[] args, HttpServer.Builder<Server> builder) throws Exception {
+    Class<? extends PhoenixMetaFactory> factoryClass = getConf().getClass(
+            QueryServices.QUERY_SERVER_META_FACTORY_ATTRIB, PhoenixMetaFactoryImpl.class,
+            PhoenixMetaFactory.class);
+    PhoenixMetaFactory factory =
+            factoryClass.getDeclaredConstructor(Configuration.class).newInstance(getConf());
+    Meta meta = factory.create(Arrays.asList(args));
+    Service service = new LocalService(meta);
+    builder.withHandler(service, getSerialization(getConf()));
   }
 
   public synchronized void stop() {
@@ -405,20 +442,32 @@ public final class QueryServer extends Configured implements Tool, Runnable {
     }
   }
 
-  private static final RemoteUserExtractorFactory DEFAULT_USER_EXTRACTOR =
-    new RemoteUserExtractorFactory.RemoteUserExtractorFactoryImpl();
-
   @VisibleForTesting
-  public void enableServerCustomizersIfNecessary(HttpServer.Builder<Server> builder, Configuration conf) {
+  public void enableServerCustomizersIfNecessary(HttpServer.Builder<Server> builder,
+                                                 Configuration conf, AvaticaServerConfiguration avaticaServerConfiguration) {
     if (conf.getBoolean(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED,
             QueryServicesOptions.DEFAULT_QUERY_SERVER_CUSTOMIZERS_ENABLED)) {
-      builder.withServerCustomizers(createServerCustomizers(conf), Server.class);
+      builder.withServerCustomizers(createServerCustomizers(conf, avaticaServerConfiguration), Server.class);
     }
   }
 
+  @VisibleForTesting
+  public AvaticaServerConfiguration enableCustomAuth(HttpServer.Builder<Server> builder,
+                                                     Configuration conf, UserGroupInformation ugi) {
+    AvaticaServerConfiguration avaticaServerConfiguration = createAvaticaServerConfig(conf, ugi);
+    builder.withCustomAuthentication(avaticaServerConfiguration);
+    return avaticaServerConfiguration;
+  }
+
+  private static final RemoteUserExtractorFactory DEFAULT_USER_EXTRACTOR =
+    new RemoteUserExtractorFactory.RemoteUserExtractorFactoryImpl();
+
   private static final ServerCustomizersFactory DEFAULT_SERVER_CUSTOMIZERS =
     new ServerCustomizersFactory.ServerCustomizersFactoryImpl();
 
+  private static final AvaticaServerConfigurationFactory DEFAULT_SERVER_CONFIG =
+    new AvaticaServerConfigurationFactory.AvaticaServerConfigurationFactoryImpl();
+
   @VisibleForTesting
   RemoteUserExtractor createRemoteUserExtractor(Configuration conf) {
     RemoteUserExtractorFactory factory =
@@ -427,16 +476,22 @@ public final class QueryServer extends Configured implements Tool, Runnable {
   }
 
   @VisibleForTesting
-  List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+  List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf, AvaticaServerConfiguration avaticaServerConfiguration) {
     ServerCustomizersFactory factory =
       InstanceResolver.getSingleton(ServerCustomizersFactory.class, DEFAULT_SERVER_CUSTOMIZERS);
-    return factory.createServerCustomizers(conf);
+    return factory.createServerCustomizers(conf, avaticaServerConfiguration);
+  }
+
+  @VisibleForTesting
+  AvaticaServerConfiguration createAvaticaServerConfig(Configuration conf, UserGroupInformation ugi) {
+    AvaticaServerConfigurationFactory factory =
+            InstanceResolver.getSingleton(AvaticaServerConfigurationFactory.class, DEFAULT_SERVER_CONFIG);
+    return factory.getAvaticaServerConfiguration(conf, ugi);
   }
 
   /**
    * Use the correctly way to extract end user.
    */
-
   static class PhoenixRemoteUserExtractor implements RemoteUserExtractor{
     private final HttpQueryStringParameterRemoteUserExtractor paramRemoteUserExtractor;
     private final HttpRequestRemoteUserExtractor requestRemoteUserExtractor;
@@ -477,7 +532,7 @@ public final class QueryServer extends Configured implements Tool, Runnable {
   /**
    * Callback to run the Avatica server action as the remote (proxy) user instead of the server.
    */
-  static class PhoenixDoAsCallback implements DoAsRemoteUserCallback {
+  public static class PhoenixDoAsCallback implements DoAsRemoteUserCallback {
     private final UserGroupInformation serverUgi;
     private final LoadingCache<String,UserGroupInformation> ugiCache;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
index 462cd5d..942660a 100644
--- a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.queryserver.server;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.ServerCustomizer;
 import org.apache.hadoop.conf.Configuration;
 import org.eclipse.jetty.server.Server;
@@ -32,9 +33,10 @@ public interface ServerCustomizersFactory {
     /**
      * Creates a list of customizers that will customize the server.
      * @param conf Configuration to use
+     * @param avaticaServerConfiguration to use in case custom-auth is enabled
      * @return List of server suctomizers
      */
-    List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf);
+    List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf, AvaticaServerConfiguration avaticaServerConfiguration);
 
     /**
      * Factory that creates an empty list of customizers.
@@ -42,7 +44,8 @@ public interface ServerCustomizersFactory {
     class ServerCustomizersFactoryImpl implements ServerCustomizersFactory {
         private static final List<ServerCustomizer<Server>> EMPTY_LIST = Collections.emptyList();
         @Override
-        public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+        public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf,
+                                                                      AvaticaServerConfiguration avaticaServerConfiguration) {
             return EMPTY_LIST;
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
new file mode 100644
index 0000000..20bc868
--- /dev/null
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
@@ -0,0 +1,20 @@
+package org.apache.phoenix.queryserver.server;
+
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class CustomAvaticaServerConfigurationTest {
+    @Test
+    public void testDefaultFactory() throws IOException {
+        QueryServer queryServer = new QueryServer();
+        UserGroupInformation ugi = queryServer.getUserGroupInformation();
+        // the default factory creates null object
+        AvaticaServerConfiguration avaticaServerConfiguration = queryServer.createAvaticaServerConfig(new Configuration(), ugi);
+        Assert.assertNull(avaticaServerConfiguration);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/QueryServerConfigurationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/QueryServerConfigurationTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/QueryServerConfigurationTest.java
index f2a1022..d01d2ea 100644
--- a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/QueryServerConfigurationTest.java
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/QueryServerConfigurationTest.java
@@ -20,10 +20,12 @@ package org.apache.phoenix.queryserver.server;
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.DoAsRemoteUserCallback;
 import org.apache.calcite.avatica.server.HttpServer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.phoenix.query.QueryServices;
 import org.junit.Before;
 import org.junit.Rule;
@@ -39,34 +41,52 @@ public class QueryServerConfigurationTest {
 
   private HttpServer.Builder builder;
   private QueryServer queryServer;
+  private UserGroupInformation ugi;
 
   @Before
   public void setup() throws IOException {
-    File keytabFile = testFolder.newFile("test.keytab");
-    CONF.set(QueryServices.QUERY_SERVER_KEYTAB_FILENAME_ATTRIB, keytabFile.getAbsolutePath());
     builder = mock(HttpServer.Builder.class);
     queryServer = new QueryServer(new String[0], CONF);
+    ugi = queryServer.getUserGroupInformation();
   }
 
   @Test
   public void testSpnegoEnabled() throws IOException {
+    setupKeytabForSpnego();
     // SPENEGO settings will be provided to the builder when enabled
     doReturn(builder).when(builder).withSpnego(anyString(), any(String[].class));
     configureAndVerifyImpersonation(builder, false);
     // A keytab file will also be provided for automatic login
     verify(builder).withAutomaticLogin(any(File.class));
+    verify(builder, never()).withCustomAuthentication(any(AvaticaServerConfiguration.class));
   }
 
   @Test
   public void testSpnegoDisabled() throws IOException {
+    setupKeytabForSpnego();
     configureAndVerifyImpersonation(builder, true);
     verify(builder, never()).withSpnego(anyString(), any(String[].class));
     verify(builder, never()).withAutomaticLogin(any(File.class));
+    verify(builder, never()).withCustomAuthentication(any(AvaticaServerConfiguration.class));
+  }
+
+  @Test
+  public void testCustomServerConfiguration() {
+    queryServer.enableCustomAuth(builder, CONF, ugi);
+    verify(builder).withCustomAuthentication(any(AvaticaServerConfiguration.class));
+    verify(builder, never()).withSpnego(anyString(), any(String[].class));
+    verify(builder, never()).withAutomaticLogin(any(File.class));
+    verify(builder, never()).withImpersonation(any(DoAsRemoteUserCallback.class));
+  }
+
+  private void setupKeytabForSpnego() throws IOException {
+    File keytabFile = testFolder.newFile("test.keytab");
+    CONF.set(QueryServices.QUERY_SERVER_KEYTAB_FILENAME_ATTRIB, keytabFile.getAbsolutePath());
   }
 
   private void configureAndVerifyImpersonation(HttpServer.Builder builder, boolean disableSpnego)
       throws IOException {
-    queryServer.configureClientAuthentication(builder, disableSpnego);
+    queryServer.configureClientAuthentication(builder, disableSpnego, ugi);
     verify(builder).withImpersonation(any(DoAsRemoteUserCallback.class));
   }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0639a742/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
index 45fec37..93e1e37 100644
--- a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.ServerCustomizer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.query.QueryServices;
@@ -45,14 +46,16 @@ public class ServerCustomizersTest {
     @Test
     public void testDefaultFactory() {
         QueryServer queryServer = new QueryServer();
+        AvaticaServerConfiguration avaticaServerConfiguration = null;
         // the default factory creates an empty list of server customizers
         List<ServerCustomizer<Server>> customizers =
-            queryServer.createServerCustomizers(new Configuration());
+                queryServer.createServerCustomizers(new Configuration(), avaticaServerConfiguration);
         Assert.assertEquals(0, customizers.size());
     }
 
     @Test
     public void testUseProvidedCustomizers() {
+        AvaticaServerConfiguration avaticaServerConfiguration = null;
         final List<ServerCustomizer<Server>> expected =
             Collections.<ServerCustomizer<Server>> singletonList(new ServerCustomizer<Server>() {
               @Override
@@ -63,25 +66,27 @@ public class ServerCustomizersTest {
         // Register the server customizer list
         InstanceResolver.getSingleton(ServerCustomizersFactory.class, new ServerCustomizersFactory() {
             @Override
-            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf,
+                                                                          AvaticaServerConfiguration avaticaServerConfiguration) {
                 return expected;
             }
         });
         Configuration conf = new Configuration(false);
         conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
         QueryServer queryServer = new QueryServer();
-        List<ServerCustomizer<Server>> actual = queryServer.createServerCustomizers(conf);
+        List<ServerCustomizer<Server>> actual = queryServer.createServerCustomizers(conf, avaticaServerConfiguration);
         Assert.assertEquals("Customizers are different", expected, actual);
     }
 
     @Test
     @SuppressWarnings("unchecked")
     public void testEnableCustomizers() {
+        AvaticaServerConfiguration avaticaServerConfiguration = null;
         HttpServer.Builder builder = mock(HttpServer.Builder.class);
         Configuration conf = new Configuration(false);
         conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
         QueryServer queryServer = new QueryServer();
-        queryServer.enableServerCustomizersIfNecessary(builder, conf);
+        queryServer.enableServerCustomizersIfNecessary(builder, conf, avaticaServerConfiguration);
         verify(builder).withServerCustomizers(anyList(), any(Class.class));
     }
 }
\ No newline at end of file


[36/50] [abbrv] phoenix git commit: PHOENIX-3178 Row count incorrect for UPSERT SELECT when auto commit is false

Posted by ja...@apache.org.
PHOENIX-3178 Row count incorrect for UPSERT SELECT when auto commit is false


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

Branch: refs/heads/omid2
Commit: a3e08104aa6d7e907c12950b2a6eb7bc9d9fb7e5
Parents: b0cc455
Author: s.kadam <s....@gus.com>
Authored: Tue Aug 28 15:44:56 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Wed Aug 29 00:21:45 2018 -0700

----------------------------------------------------------------------
 .../end2end/UpsertSelectAutoCommitIT.java       | 31 ++++++++++++++++++--
 .../apache/phoenix/compile/UpsertCompiler.java  | 10 +++++--
 2 files changed, 37 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3e08104/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index 38d48d6..3966f15 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -151,8 +151,7 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         stmt.executeUpdate();
         conn.commit();
     }
-    
-    
+
     @Test
     public void testUpsertSelectDoesntSeeUpsertedData() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -201,4 +200,32 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         connection.close();
     }
 
+    @Test
+    public void testRowCountWithNoAutoCommitOnUpsertSelect() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        String tableName = generateUniqueName();
+
+        conn.createStatement().execute("CREATE SEQUENCE "+ tableName);
+        conn.createStatement().execute(
+                "CREATE TABLE " + tableName + " (pk INTEGER PRIMARY KEY, val INTEGER)");
+
+        conn.createStatement().execute(
+                "UPSERT INTO " + tableName + " VALUES (NEXT VALUE FOR keys,1)");
+        conn.commit();
+        for (int i=0; i<6; i++) {
+            Statement stmt = conn.createStatement();
+            int upsertCount = stmt.executeUpdate(
+                    "UPSERT INTO " + tableName + " SELECT NEXT VALUE FOR keys, val FROM "
+                            + tableName);
+            conn.commit();
+            assertEquals((int)Math.pow(2, i), upsertCount);
+        }
+        conn.close();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3e08104/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 9d75bba..d0dd2cf 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
@@ -185,6 +185,7 @@ public class UpsertCompiler {
                     QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
         int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
         boolean isAutoCommit = connection.getAutoCommit();
+        int sizeOffset = 0;
         int numSplColumns =
                 (tableRef.getTable().isMultiTenant() ? 1 : 0)
                         + (tableRef.getTable().getViewIndexId() != null ? 1 : 0);
@@ -249,8 +250,13 @@ public class UpsertCompiler {
                     mutation.clear();
                 }
             }
-            // If auto commit is true, this last batch will be committed upon return
-            return new MutationState(tableRef, mutation, rowCount / batchSize * batchSize, maxSize, maxSizeBytes, connection);
+
+            if (isAutoCommit) {
+                // If auto commit is true, this last batch will be committed upon return
+                sizeOffset = rowCount / batchSize * batchSize;
+            }
+            return new MutationState(tableRef, mutation, sizeOffset, maxSize,
+                    maxSizeBytes, connection);
         }
     }
 


[16/50] [abbrv] phoenix git commit: PHOENIX-4824 Update BRANCH_NAMES in dev/test-patch.properties

Posted by ja...@apache.org.
PHOENIX-4824 Update BRANCH_NAMES in dev/test-patch.properties


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

Branch: refs/heads/omid2
Commit: bf6db8f4d57418406929e553f6be3dca77778d78
Parents: b37b7d7
Author: Thomas D'Silva <td...@apache.org>
Authored: Sat Jul 28 11:49:08 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Sat Jul 28 14:23:23 2018 -0700

----------------------------------------------------------------------
 dev/test-patch.properties | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bf6db8f4/dev/test-patch.properties
----------------------------------------------------------------------
diff --git a/dev/test-patch.properties b/dev/test-patch.properties
index ea8d75d..b3dc46f 100644
--- a/dev/test-patch.properties
+++ b/dev/test-patch.properties
@@ -27,7 +27,7 @@ MAX_LINE_LENGTH=100
 # All supported branches for testing with precommit build
 # be sure to consider branch name prefixes in the order, ie, 4.x should appear
 # before 4 since the latter is a prefix
-BRANCH_NAMES="4.x-HBase-0.98 4.x-HBase-1.1 4.x-HBase-1.2 master"
+BRANCH_NAMES="4.x-HBase-1.2 4.x-HBase-1.3 4.x-HBase-1.4 master"
 
 
 # All supported Hadoop versions that we want to test the compilation with


[34/50] [abbrv] phoenix git commit: PHOENIX-4864 Fix NullPointerException while Logging some DDL Statements

Posted by ja...@apache.org.
PHOENIX-4864 Fix NullPointerException while Logging some DDL Statements


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

Branch: refs/heads/omid2
Commit: 46f50d29435e29d35dc5cca0f9c68711b22c8308
Parents: 79e8337
Author: Ashutosh Parekh <pa...@gmail.com>
Authored: Thu Aug 23 12:11:23 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Fri Aug 24 17:35:46 2018 -0700

----------------------------------------------------------------------
 .../monitoring/PhoenixLoggingMetricsIT.java     | 42 +++++++++++++++++++-
 .../jdbc/LoggingPhoenixPreparedStatement.java   |  5 ++-
 .../phoenix/jdbc/LoggingPhoenixStatement.java   |  5 ++-
 3 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/46f50d29/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
index 7e56902..5d5524c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.monitoring;
 import com.google.common.collect.Maps;
 import org.apache.phoenix.jdbc.LoggingPhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixMetricsLog;
+import org.apache.phoenix.jdbc.LoggingPhoenixResultSet;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -69,6 +70,35 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     }
 
     @Test
+    public void testResultSetTypeForQueries() throws Exception {
+        String tableName3 = generateUniqueName();
+
+        String create = "CREATE TABLE " + tableName3 + " (K INTEGER PRIMARY KEY)";
+        assertTrue(executeAndGetResultSet(create) == null);
+
+        String upsert = "UPSERT INTO " + tableName3 + " VALUES (42)";
+        assertTrue(executeAndGetResultSet(upsert) == null);
+
+        String select = "SELECT * FROM " + tableName3;
+        assertTrue(executeAndGetResultSet(select) instanceof LoggingPhoenixResultSet);
+
+        String createView = "CREATE VIEW TEST_VIEW (K INTEGER) AS SELECT * FROM " + tableName3;
+        assertTrue(executeAndGetResultSet(createView) == null);
+
+        String createIndex = "CREATE INDEX TEST_INDEX ON " + tableName3 + " (K)";
+        assertTrue(executeAndGetResultSet(createIndex) == null);
+
+        String dropIndex = "DROP INDEX TEST_INDEX ON " + tableName3;
+        assertTrue(executeAndGetResultSet(dropIndex) == null);
+
+        String dropView = "DROP VIEW TEST_VIEW";
+        assertTrue(executeAndGetResultSet(dropView) == null);
+
+        String dropTable = "DROP TABLE " + tableName3;
+        assertTrue(executeAndGetResultSet(dropTable) == null);
+    }
+
+    @Test
     public void testPhoenixMetricsLoggedOnCommit() throws Exception {
         // run SELECT to verify read metrics are logged
         String query = "SELECT * FROM " + tableName1;
@@ -134,12 +164,22 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
                 mutationReadMetricsMap.size() == 0);
     }
 
+    private ResultSet executeAndGetResultSet(String query) throws Exception {
+        Statement stmt = loggedConn.createStatement();
+        stmt.execute(query);
+        return stmt.getResultSet();
+    }
+
     private void verifyQueryLevelMetricsLogging(String query) throws SQLException {
         Statement stmt = loggedConn.createStatement();
         ResultSet rs = stmt.executeQuery(query);
+        assertTrue(rs instanceof LoggingPhoenixResultSet);
+        int rowsRetrievedCounter = 0;
         while (rs.next()) {
+            rowsRetrievedCounter++;
         }
         rs.close();
+        assertTrue(rowsRetrievedCounter == NUM_ROWS);
         assertTrue("Read metrics for not found for " + tableName1,
                 requestReadMetricsMap.get(tableName1).size() > 0);
         assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
@@ -192,4 +232,4 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
             }
         });
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/46f50d29/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
index 952e3fd..dbeea0d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
@@ -45,7 +45,10 @@ public class LoggingPhoenixPreparedStatement extends DelegatePreparedStatement {
 
     @Override
     public ResultSet getResultSet() throws SQLException {
-        return new LoggingPhoenixResultSet(super.getResultSet(), phoenixMetricsLog, sql);
+        // Re-use the cached ResultSet value since call to getResultSet() is not idempotent
+        ResultSet resultSet = super.getResultSet();
+        return (resultSet == null) ? null : new LoggingPhoenixResultSet(resultSet,
+                phoenixMetricsLog, sql);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/46f50d29/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
index f9abe6a..de33893 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
@@ -51,7 +51,10 @@ public class LoggingPhoenixStatement extends DelegateStatement {
 
     @Override
     public ResultSet getResultSet() throws SQLException {
-        return new LoggingPhoenixResultSet(super.getResultSet(), phoenixMetricsLog, this.sql);
+        // Re-use the cached ResultSet value since call to getResultSet() is not idempotent
+        ResultSet resultSet = super.getResultSet();
+        return (resultSet == null) ? null : new LoggingPhoenixResultSet(resultSet,
+                phoenixMetricsLog, sql);
     }
     
     @Override


[17/50] [abbrv] phoenix git commit: PHOENIX-4825 Replace usage of HBase Base64 implementation with java.util.Base64

Posted by ja...@apache.org.
PHOENIX-4825 Replace usage of HBase Base64 implementation with java.util.Base64


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

Branch: refs/heads/omid2
Commit: 6f5926b6b1f3d89b7283a5d030d6f46533dc0d39
Parents: bf6db8f
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Jul 30 13:56:47 2018 -0700
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Jul 30 13:56:47 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/QueryMoreIT.java |  7 +++--
 .../phoenix/mapreduce/CsvBulkImportUtil.java    |  8 ++++--
 .../util/PhoenixConfigurationUtil.java          |  7 +++--
 .../apache/phoenix/schema/types/PVarbinary.java |  4 +--
 .../phoenix/util/csv/CsvUpsertExecutor.java     |  4 +--
 .../phoenix/util/json/JsonUpsertExecutor.java   |  4 +--
 .../util/AbstractUpsertExecutorTest.java        | 12 ++++----
 .../util/TenantIdByteConversionTest.java        | 30 ++++++++++++++++----
 8 files changed, 50 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
index 04272fa..528fe7f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
@@ -31,12 +31,13 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Base64;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
@@ -278,7 +279,7 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
                 values[i] = rs.getObject(i + 1);
             }
             conn = getTenantSpecificConnection(tenantId);
-            pkIds.add(Base64.encodeBytes(PhoenixRuntime.encodeColumnValues(conn, tableOrViewName.toUpperCase(), values, columns)));
+            pkIds.add(Bytes.toString(Base64.getEncoder().encode(PhoenixRuntime.encodeColumnValues(conn, tableOrViewName.toUpperCase(), values, columns))));
         }
         return pkIds.toArray(new String[pkIds.size()]);
     }
@@ -296,7 +297,7 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
         PreparedStatement stmt = conn.prepareStatement(query);
         int bindCounter = 1;
         for (int i = 0; i < cursorIds.length; i++) {
-            Object[] pkParts = PhoenixRuntime.decodeColumnValues(conn, tableName.toUpperCase(), Base64.decode(cursorIds[i]), columns);
+            Object[] pkParts = PhoenixRuntime.decodeColumnValues(conn, tableName.toUpperCase(), Base64.getDecoder().decode(cursorIds[i]), columns);
             for (int j = 0; j < pkParts.length; j++) {
                 stmt.setObject(bindCounter++, pkParts[j]);
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
index ff9ff72..bf5a538 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
@@ -17,9 +17,11 @@
  */
 package org.apache.phoenix.mapreduce;
 
+import java.util.Base64;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
@@ -68,7 +70,7 @@ public class CsvBulkImportUtil {
 
     @VisibleForTesting
     static void setChar(Configuration conf, String confKey, char charValue) {
-        conf.set(confKey, Base64.encodeBytes(Character.toString(charValue).getBytes()));
+        conf.set(confKey, Bytes.toString(Base64.getEncoder().encode(Character.toString(charValue).getBytes())));
     }
 
     @VisibleForTesting
@@ -77,7 +79,7 @@ public class CsvBulkImportUtil {
         if (strValue == null) {
             return null;
         }
-        return new String(Base64.decode(strValue)).charAt(0);
+        return new String(Base64.getDecoder().decode(strValue)).charAt(0);
     }
 
     public static Path getOutputPath(Path outputdir, String tableName) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
index f3f0415..3b63f66 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
@@ -24,6 +24,7 @@ import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FO
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.Base64;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -35,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.db.DBInputFormat.NullDBWritable;
 import org.apache.hadoop.mapreduce.lib.db.DBWritable;
@@ -506,14 +507,14 @@ public final class PhoenixConfigurationUtil {
 
     public static byte[] getIndexMaintainers(final Configuration configuration){
         Preconditions.checkNotNull(configuration);
-        return Base64.decode(configuration.get(INDEX_MAINTAINERS));
+        return Base64.getDecoder().decode(configuration.get(INDEX_MAINTAINERS));
     }
     
     public static void setIndexMaintainers(final Configuration configuration,
             final ImmutableBytesWritable indexMetaDataPtr) {
         Preconditions.checkNotNull(configuration);
         Preconditions.checkNotNull(indexMetaDataPtr);
-        configuration.set(INDEX_MAINTAINERS, Base64.encodeBytes(indexMetaDataPtr.get()));
+        configuration.set(INDEX_MAINTAINERS,Bytes.toString(Base64.getEncoder().encode(indexMetaDataPtr.get())));
     }
     
     public static void setDisableIndexes(Configuration configuration, String indexName) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
index b3ce57a..e165a9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
@@ -19,8 +19,8 @@ package org.apache.phoenix.schema.types;
 
 import java.sql.Types;
 import java.text.Format;
+import java.util.Base64;
 
-import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
@@ -131,7 +131,7 @@ public class PVarbinary extends PBinaryBase {
         if (value == null || value.length() == 0) {
             return null;
         }
-        Object object = Base64.decode(value);
+        Object object = Base64.getDecoder().decode(value);
         if (object == null) { throw newIllegalDataException(
                 "Input: [" + value + "]  is not base64 encoded"); }
         return object;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
index cd40b44..4f98ada 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
@@ -21,13 +21,13 @@ import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Types;
+import java.util.Base64;
 import java.util.List;
 import java.util.Properties;
 
 import javax.annotation.Nullable;
 
 import org.apache.commons.csv.CSVRecord;
-import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.expression.function.EncodeFormat;
 import org.apache.phoenix.query.QueryServices;
@@ -189,7 +189,7 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
                 Object object = null;
                 switch (format) {
                     case BASE64:
-                        object = Base64.decode(input);
+                        object = Base64.getDecoder().decode(input);
                         if (object == null) { throw new IllegalDataException(
                                 "Input: [" + input + "]  is not base64 encoded"); }
                         break;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
index ffa797d..867a4cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
@@ -22,13 +22,13 @@ import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Types;
+import java.util.Base64;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
 import javax.annotation.Nullable;
 
-import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.expression.function.EncodeFormat;
 import org.apache.phoenix.query.QueryServices;
@@ -212,7 +212,7 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
             Object object = null;
             switch (format) {
                 case BASE64:
-                    object = Base64.decode(input.toString());
+                    object = Base64.getDecoder().decode(input.toString());
                     if (object == null) { throw new IllegalDataException(
                             "Input: [" + input + "]  is not base64 encoded"); }
                     break;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
index 2b2544d..02bf548 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
@@ -30,10 +30,10 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Types;
 import java.util.Arrays;
+import java.util.Base64;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.QueryServices;
@@ -81,7 +81,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
     @Test
     public void testExecute() throws Exception {
         byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
-        String encodedBinaryData = Base64.encodeBytes(binaryData);
+        String encodedBinaryData = Bytes.toString(Base64.getEncoder().encode(binaryData));
         getUpsertExecutor().execute(createRecord(123L, "NameValue", 42,
                 Arrays.asList(1, 2, 3), true, encodedBinaryData));
 
@@ -110,7 +110,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
     @Test
     public void testExecute_TooManyFields() throws Exception {
         byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
-        String encodedBinaryData = Base64.encodeBytes(binaryData);
+        String encodedBinaryData = Bytes.toString(Base64.getEncoder().encode(binaryData));
         R recordWithTooManyFields = createRecord(123L, "NameValue", 42, Arrays.asList(1, 2, 3),
                 true, encodedBinaryData, "garbage");
         getUpsertExecutor().execute(recordWithTooManyFields);
@@ -131,7 +131,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
     @Test
     public void testExecute_NullField() throws Exception {
         byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
-        String encodedBinaryData = Base64.encodeBytes(binaryData);
+        String encodedBinaryData = Bytes.toString(Base64.getEncoder().encode(binaryData));
         getUpsertExecutor().execute(createRecord(123L, "NameValue", null,
                 Arrays.asList(1, 2, 3), false, encodedBinaryData));
 
@@ -151,7 +151,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
     @Test
     public void testExecute_InvalidType() throws Exception {
         byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
-        String encodedBinaryData = Base64.encodeBytes(binaryData);
+        String encodedBinaryData = Bytes.toString(Base64.getEncoder().encode(binaryData));
         R recordWithInvalidType = createRecord(123L, "NameValue", "ThisIsNotANumber",
                 Arrays.asList(1, 2, 3), true, encodedBinaryData);
         getUpsertExecutor().execute(recordWithInvalidType);
@@ -163,7 +163,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
     @Test
     public void testExecute_InvalidBoolean() throws Exception {
         byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
-        String encodedBinaryData = Base64.encodeBytes(binaryData);
+        String encodedBinaryData = Bytes.toString(Base64.getEncoder().encode(binaryData));
         R csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,NotABoolean,"+encodedBinaryData);
         getUpsertExecutor().execute(csvRecordWithInvalidType);
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f5926b6/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
index fb70d22..3ef9230 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TenantIdByteConversionTest.java
@@ -22,20 +22,40 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.fail;
 
 import java.sql.SQLException;
-import org.apache.hadoop.hbase.util.Base64;
+import java.util.Base64;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.phoenix.schema.*;
-import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBinary;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PFloat;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PSmallint;
+import org.apache.phoenix.schema.types.PTinyint;
+import org.apache.phoenix.schema.types.PUnsignedDouble;
+import org.apache.phoenix.schema.types.PUnsignedFloat;
+import org.apache.phoenix.schema.types.PUnsignedInt;
+import org.apache.phoenix.schema.types.PUnsignedLong;
+import org.apache.phoenix.schema.types.PUnsignedSmallint;
+import org.apache.phoenix.schema.types.PUnsignedTinyint;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Lists;
-import org.mockito.Mockito;
 
 /*Test the getTenantIdBytes method in ScanUtil*/
 @RunWith(Parameterized.class)
@@ -201,7 +221,7 @@ public class TenantIdByteConversionTest {
 
         //Binary
         byte[] bytes = new byte[] {0, 1, 2, 3};
-        String byteString = new String( Base64.encodeBytes(bytes) );
+        String byteString = new String( Base64.getEncoder().encode(bytes) );
         testCases.add(new Object[] {
                 getDataSchema(PBinary.INSTANCE, SortOrder.getDefault()),
                 false,


[39/50] [abbrv] phoenix git commit: PHOENIX-4791 Array elements are nullified with joins

Posted by ja...@apache.org.
PHOENIX-4791 Array elements are nullified with joins


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

Branch: refs/heads/omid2
Commit: 5900ec8dd8316bb7e9d36611587c051be32a2092
Parents: bef8d7d
Author: Gerald Sangudi <gs...@23andme.com>
Authored: Wed Aug 22 16:59:12 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Sep 6 10:30:51 2018 -0700

----------------------------------------------------------------------
 .../ProjectArrayElemAfterHashJoinIT.java        | 177 +++++++++++++++++++
 .../coprocessor/HashJoinRegionScanner.java      |  69 ++++++--
 .../NonAggregateRegionScannerFactory.java       |   5 +-
 .../phoenix/iterate/RegionScannerFactory.java   |   7 +-
 4 files changed, 243 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5900ec8d/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProjectArrayElemAfterHashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProjectArrayElemAfterHashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProjectArrayElemAfterHashJoinIT.java
new file mode 100644
index 0000000..170eb69
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProjectArrayElemAfterHashJoinIT.java
@@ -0,0 +1,177 @@
+/*
+ * 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.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.junit.Test;
+
+public class ProjectArrayElemAfterHashJoinIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testSalted() throws Exception {
+
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        try {
+            String table = createSalted(conn);
+            testTable(conn, table);
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testUnsalted() throws Exception {
+
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        try {
+            String table = createUnsalted(conn);
+            testTable(conn, table);
+        } finally {
+            conn.close();
+        }
+    }
+
+    private void testTable(Connection conn, String table) throws Exception {
+
+        verifyExplain(conn, table, false, false);
+        verifyExplain(conn, table, false, true);
+        verifyExplain(conn, table, true, false);
+        verifyExplain(conn, table, true, true);
+
+        verifyResults(conn, table, false, false);
+        verifyResults(conn, table, false, true);
+        verifyResults(conn, table, true, false);
+        verifyResults(conn, table, true, true);
+    }
+
+    private String createSalted(Connection conn) throws Exception {
+
+        String table = "SALTED_" + generateUniqueName();
+        String create = "CREATE TABLE " + table + " ("
+            + " id INTEGER NOT NULL,"
+            + " vals TINYINT[],"
+            + " CONSTRAINT pk PRIMARY KEY (id)"
+            + ") SALT_BUCKETS = 4";
+
+        conn.createStatement().execute(create);
+        return table;
+    }
+
+    private String createUnsalted(Connection conn) throws Exception {
+
+        String table = "UNSALTED_" + generateUniqueName();
+        String create = "CREATE TABLE " + table + " ("
+            + " id INTEGER NOT NULL,"
+            + " vals TINYINT[],"
+            + " CONSTRAINT pk PRIMARY KEY (id)"
+            + ")";
+
+        conn.createStatement().execute(create);
+        return table;
+    }
+
+    private String getQuery(String table, boolean fullArray, boolean hashJoin) {
+
+        String query = "SELECT id, vals[1] v1, vals[2] v2, vals[3] v3, vals[4] v4"
+            + (fullArray ? ", vals" : "")
+            + " FROM " + table
+            + " WHERE id IN "
+            + (hashJoin ? "(SELECT 1)" : "(1, 2, 3)")
+            ;
+
+        return query;
+    }
+
+    private void verifyExplain(Connection conn, String table, boolean fullArray, boolean hashJoin)
+        throws Exception {
+
+        String query = "EXPLAIN " + getQuery(table, fullArray, hashJoin);
+        Statement stmt = conn.createStatement();
+        ResultSet rs = stmt.executeQuery(query);
+
+        try {
+            String plan = QueryUtil.getExplainPlan(rs);
+            assertTrue(plan != null);
+            assertTrue(fullArray || plan.contains("SERVER ARRAY ELEMENT PROJECTION"));
+            assertTrue(hashJoin == plan.contains("JOIN"));
+        } finally {
+            rs.close();
+        }
+    }
+
+    private void verifyResults(Connection conn, String table, boolean fullArray, boolean hashJoin)
+        throws Exception {
+
+        String upsert = "UPSERT INTO " + table + "(id, vals)"
+            + " VALUES(1, ARRAY[10, 20, 30, 40, 50])";
+        PreparedStatement upsertStmt = conn.prepareStatement(upsert);
+        upsertStmt.execute();
+        conn.commit();
+
+        String query = getQuery(table, fullArray, hashJoin);
+        Statement stmt = conn.createStatement();
+        ResultSet rs = stmt.executeQuery(query);
+
+        try {
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt("id"));
+            assertEquals(10, rs.getInt("v1"));
+            assertEquals(20, rs.getInt("v2"));
+            assertEquals(30, rs.getInt("v3"));
+            assertEquals(40, rs.getInt("v4"));
+
+            if (fullArray) {
+                java.sql.Array array = rs.getArray("vals");
+                assertTrue(array != null);
+                Object obj = array.getArray();
+                assertTrue(obj != null);
+                assertTrue(obj.getClass().isArray());
+                assertEquals(5, java.lang.reflect.Array.getLength(obj));
+            }
+
+            assertFalse(rs.next());
+        } finally {
+            rs.close();
+        }
+    }
+
+    private void dropTable(Connection conn, String table) throws Exception {
+
+        String drop = "DROP TABLE " + table;
+        Statement stmt = conn.createStatement();
+        stmt.execute(drop);
+        stmt.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5900ec8d/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
index 59f844d..d82aaba 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -18,10 +18,12 @@
 package org.apache.phoenix.coprocessor;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -37,12 +39,15 @@ import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.execute.TupleProjector.ProjectedValueTuple;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.iterate.RegionScannerFactory;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.tuple.PositionBasedResultTuple;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -66,9 +71,27 @@ public class HashJoinRegionScanner implements RegionScanner {
     private ValueBitSet[] tempSrcBitSet;
     private final boolean useQualifierAsListIndex;
     private final boolean useNewValueColumnQualifier;
-    
+    private final boolean addArrayCell;
+
+    @SuppressWarnings("unchecked")
+    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector,
+                                 HashJoinInfo joinInfo, ImmutableBytesPtr tenantId,
+                                 RegionCoprocessorEnvironment env, boolean useQualifierAsIndex,
+                                 boolean useNewValueColumnQualifier)
+        throws IOException {
+
+        this(env, scanner, null, null, projector, joinInfo,
+             tenantId, useQualifierAsIndex, useNewValueColumnQualifier);
+    }
+
     @SuppressWarnings("unchecked")
-    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesPtr tenantId, RegionCoprocessorEnvironment env, boolean useQualifierAsIndex, boolean useNewValueColumnQualifier) throws IOException {
+    public HashJoinRegionScanner(RegionCoprocessorEnvironment env, RegionScanner scanner,
+                                 final Set<KeyValueColumnExpression> arrayKVRefs,
+                                 final Expression[] arrayFuncRefs, TupleProjector projector,
+                                 HashJoinInfo joinInfo, ImmutableBytesPtr tenantId,
+                                 boolean useQualifierAsIndex, boolean useNewValueColumnQualifier)
+        throws IOException {
+
         this.env = env;
         this.scanner = scanner;
         this.projector = projector;
@@ -103,7 +126,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                         Bytes.toLong(ByteUtil.copyKeyBytesIfNecessary(joinId)));
                 throw new DoNotRetryIOException(cause.getMessage(), cause);
             }
-                
+
             hashCaches[i] = hashCache;
             tempSrcBitSet[i] = ValueBitSet.newInstance(joinInfo.getSchemas()[i]);
         }
@@ -113,16 +136,21 @@ public class HashJoinRegionScanner implements RegionScanner {
         }
         this.useQualifierAsListIndex = useQualifierAsIndex;
         this.useNewValueColumnQualifier = useNewValueColumnQualifier;
+        this.addArrayCell = (arrayFuncRefs != null && arrayFuncRefs.length > 0 &&
+                             arrayKVRefs != null && arrayKVRefs.size() > 0);
     }
 
     private void processResults(List<Cell> result, boolean hasBatchLimit) throws IOException {
         if (result.isEmpty())
             return;
         Tuple tuple = useQualifierAsListIndex ? new PositionBasedResultTuple(result) : new ResultTuple(Result.create(result));
+        boolean projected = false;
+
         // For backward compatibility. In new versions, HashJoinInfo.forceProjection()
         // always returns true.
         if (joinInfo.forceProjection()) {
             tuple = projector.projectResults(tuple, useNewValueColumnQualifier);
+            projected = true;
         }
 
         // TODO: fix below Scanner.next() and Scanner.nextRaw() methods as well.
@@ -150,14 +178,15 @@ public class HashJoinRegionScanner implements RegionScanner {
                     dup *= (tempTuples[i] == null ? 1 : tempTuples[i].size());
                 }
                 for (int i = 0; i < dup; i++) {
-                    resultQueue.offer(tuple);
+                    offerResult(tuple, projected, result);
                 }
             } else {
                 KeyValueSchema schema = joinInfo.getJoinedSchema();
                 if (!joinInfo.forceProjection()) { // backward compatibility
                     tuple = projector.projectResults(tuple, useNewValueColumnQualifier);
+                    projected = true;
                 }
-                resultQueue.offer(tuple);
+                offerResult(tuple, projected, result);
                 for (int i = 0; i < count; i++) {
                     boolean earlyEvaluation = joinInfo.earlyEvaluation()[i];
                     JoinType type = joinInfo.getJoinTypes()[i];
@@ -173,7 +202,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                                 if (type == JoinType.Inner || type == JoinType.Semi) {
                                     continue;
                                 } else if (type == JoinType.Anti) {
-                                    resultQueue.offer(lhs);
+                                    offerResult(lhs, projected, result);
                                     continue;
                                 }
                             }
@@ -182,18 +211,18 @@ public class HashJoinRegionScanner implements RegionScanner {
                             Tuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
                                     lhs : TupleProjector.mergeProjectedValue(
                                             (ProjectedValueTuple) lhs, schema, tempDestBitSet,
-                                            null, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                            null, joinInfo.getSchemas()[i], tempSrcBitSet[i],
                                             joinInfo.getFieldPositions()[i], useNewValueColumnQualifier);
-                            resultQueue.offer(joined);
+                            offerResult(joined, projected, result);
                             continue;
                         }
                         for (Tuple t : tempTuples[i]) {
                             Tuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
                                     lhs : TupleProjector.mergeProjectedValue(
                                             (ProjectedValueTuple) lhs, schema, tempDestBitSet,
-                                            t, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                            t, joinInfo.getSchemas()[i], tempSrcBitSet[i],
                                             joinInfo.getFieldPositions()[i], useNewValueColumnQualifier);
-                            resultQueue.offer(joined);
+                            offerResult(joined, projected, result);
                         }
                     }
                 }
@@ -265,7 +294,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                 processResults(result, false);
                 result.clear();
             }
-            
+
             return nextInQueue(result);
         } catch (Throwable t) {
             ServerUtil.throwIOException(env.getRegion().getRegionInfo().getRegionNameAsString(), t);
@@ -309,5 +338,21 @@ public class HashJoinRegionScanner implements RegionScanner {
         return this.scanner.getBatch();
     }
 
-}
+    // PHOENIX-4791 Propagate array element cell through hash join
+    private void offerResult(Tuple tuple, boolean projected, List<Cell> result) {
+        if (!projected || !addArrayCell) {
+            resultQueue.offer(tuple);
+            return;
+        }
 
+        Cell projectedCell = tuple.getValue(0);
+        int arrayCellPosition = RegionScannerFactory.getArrayCellPosition(result);
+        Cell arrayCell = result.get(arrayCellPosition);
+
+        List<Cell> cells = new ArrayList<Cell>(2);
+        cells.add(projectedCell);
+        cells.add(arrayCell);
+        MultiKeyValueTuple multi = new MultiKeyValueTuple(cells);
+        resultQueue.offer(multi);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5900ec8d/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 cc7221e..1504a7c 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
@@ -152,8 +152,9 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
 
     final ImmutableBytesPtr tenantId = ScanUtil.getTenantId(scan);
     if (j != null) {
-      innerScanner = new HashJoinRegionScanner(innerScanner, p, j, tenantId, env, useQualifierAsIndex,
-          useNewValueColumnQualifier);
+        innerScanner = new HashJoinRegionScanner(env, innerScanner, arrayKVRefs, arrayFuncRefs,
+                                                 p, j, tenantId, useQualifierAsIndex,
+                                                 useNewValueColumnQualifier);
     }
     if (scanOffset != null) {
       innerScanner = getOffsetScanner(innerScanner, new OffsetResultIterator(

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5900ec8d/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 aed5805..b47d6b8 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
@@ -286,7 +286,7 @@ public abstract class RegionScannerFactory {
             QueryConstants.ARRAY_VALUE_COLUMN_QUALIFIER, 0,
             QueryConstants.ARRAY_VALUE_COLUMN_QUALIFIER.length, HConstants.LATEST_TIMESTAMP,
             KeyValue.Type.codeToType(rowKv.getTypeByte()), value, 0, value.length));
-        return result.size() - 1;
+        return getArrayCellPosition(result);
       }
 
       @Override
@@ -300,4 +300,9 @@ public abstract class RegionScannerFactory {
       }
     };
   }
+
+    // PHOENIX-4791 Share position of array element cell
+    public static int getArrayCellPosition(List<Cell> result) {
+        return result.size() - 1;
+    }
 }


[50/50] [abbrv] phoenix git commit: Merge branch '4.x-HBase-1.3' into omid2

Posted by ja...@apache.org.
Merge branch '4.x-HBase-1.3' into omid2


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

Branch: refs/heads/omid2
Commit: 24178207a94ba1f0900d1962896764c361a1efd6
Parents: 119212c e5379cd
Author: James Taylor <ja...@apache.org>
Authored: Mon Sep 24 08:27:43 2018 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Mon Sep 24 08:27:43 2018 -0700

----------------------------------------------------------------------
 bin/phoenix_utils.py                            |    7 +
 bin/queryserver.py                              |    4 +-
 bin/traceserver.py                              |    6 +-
 dev/test-patch.properties                       |    2 +-
 phoenix-client/pom.xml                          |    1 +
 .../StatisticsCollectionRunTrackerIT.java       |    2 +-
 .../AlterMultiTenantTableWithViewsIT.java       |  284 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |   45 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  544 ++--
 .../phoenix/end2end/AppendOnlySchemaIT.java     |    4 +-
 .../phoenix/end2end/BasePermissionsIT.java      |   66 +-
 .../end2end/BaseTenantSpecificViewIndexIT.java  |   42 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |    8 +-
 .../phoenix/end2end/ChangePermissionsIT.java    |    4 +-
 .../phoenix/end2end/ClientHashAggregateIT.java  |  208 ++
 .../org/apache/phoenix/end2end/DateTimeIT.java  |   77 +
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |   69 +-
 .../org/apache/phoenix/end2end/InListIT.java    |   55 +
 .../apache/phoenix/end2end/InstrFunctionIT.java |   35 +
 .../MigrateSystemTablesToSystemNamespaceIT.java |   51 +-
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   37 +-
 .../ProjectArrayElemAfterHashJoinIT.java        |  177 ++
 .../end2end/QueryDatabaseMetaDataIT.java        |   13 +-
 .../org/apache/phoenix/end2end/QueryMoreIT.java |   19 +-
 .../apache/phoenix/end2end/RowTimestampIT.java  |   26 +-
 .../apache/phoenix/end2end/SaltedViewIT.java    |   45 -
 .../phoenix/end2end/SplitSystemCatalogIT.java   |   80 +
 .../end2end/SplitSystemCatalogTests.java        |   28 +
 .../StatsEnabledSplitSystemCatalogIT.java       |  261 ++
 .../SystemCatalogCreationOnConnectionIT.java    |   34 +-
 .../apache/phoenix/end2end/SystemCatalogIT.java |   31 +-
 .../phoenix/end2end/TableDDLPermissionsIT.java  |    9 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |   15 +-
 .../end2end/TenantSpecificViewIndexIT.java      |   70 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   |  355 +--
 .../end2end/UpsertSelectAutoCommitIT.java       |   31 +-
 .../java/org/apache/phoenix/end2end/ViewIT.java | 1144 ++++++--
 .../phoenix/end2end/index/BaseIndexIT.java      |   43 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |   11 +-
 .../phoenix/end2end/index/DropColumnIT.java     |  117 -
 .../index/GlobalIndexOptimizationIT.java        |    2 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |    4 +-
 .../phoenix/end2end/index/IndexUsageIT.java     |    4 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |    2 +-
 .../end2end/index/MutableIndexFailureIT.java    |  124 +-
 .../MutableIndexFailureWithNamespaceIT.java     |   82 +
 .../phoenix/end2end/index/MutableIndexIT.java   |  838 +++---
 .../phoenix/end2end/index/ViewIndexIT.java      |   74 +-
 .../end2end/join/HashJoinPersistentCacheIT.java |  167 ++
 .../apache/phoenix/execute/PartialCommitIT.java |    2 +-
 .../monitoring/BasePhoenixMetricsIT.java        |  128 +
 .../monitoring/PhoenixLoggingMetricsIT.java     |  290 ++
 .../phoenix/monitoring/PhoenixMetricsIT.java    |  171 +-
 .../phoenix/query/ConnectionCachingIT.java      |  104 +
 .../SystemCatalogWALEntryFilterIT.java          |   85 +-
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |    9 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |    6 +-
 .../regionserver/IndexHalfStoreFileReader.java  |   48 +-
 .../IndexHalfStoreFileReaderGenerator.java      |  153 +-
 .../org/apache/phoenix/cache/GlobalCache.java   |   22 +-
 .../apache/phoenix/cache/ServerCacheClient.java |  118 +-
 .../org/apache/phoenix/cache/TenantCache.java   |    2 +-
 .../apache/phoenix/cache/TenantCacheImpl.java   |  209 +-
 .../ColumnNameTrackingExpressionCompiler.java   |   46 +
 .../phoenix/compile/CreateTableCompiler.java    |    7 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |    4 +-
 .../apache/phoenix/compile/FromCompiler.java    |   22 +-
 .../apache/phoenix/compile/JoinCompiler.java    |    2 +-
 .../org/apache/phoenix/compile/KeyPart.java     |    2 -
 .../phoenix/compile/ListJarsQueryPlan.java      |    2 +-
 .../apache/phoenix/compile/QueryCompiler.java   |    9 +-
 .../org/apache/phoenix/compile/ScanRanges.java  |   77 +-
 .../phoenix/compile/StatementContext.java       |   32 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    2 +-
 .../compile/TupleProjectionCompiler.java        |    4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    4 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   16 +-
 .../apache/phoenix/compile/WhereOptimizer.java  | 1306 ++++++---
 .../GroupedAggregateRegionObserver.java         |    8 -
 .../coprocessor/HashJoinRegionScanner.java      |   73 +-
 .../coprocessor/MetaDataEndpointImpl.java       | 2704 +++++++++---------
 .../phoenix/coprocessor/MetaDataProtocol.java   |   56 +-
 .../PhoenixMetaDataCoprocessorHost.java         |    2 +-
 .../coprocessor/ServerCachingEndpointImpl.java  |    2 +-
 .../apache/phoenix/coprocessor/TableInfo.java   |   79 +
 .../coprocessor/TableViewFinderResult.java      |   48 +
 .../apache/phoenix/coprocessor/ViewFinder.java  |  144 +
 .../coprocessor/WhereConstantParser.java        |  106 +
 .../coprocessor/generated/MetaDataProtos.java   |  969 ++++++-
 .../coprocessor/generated/PTableProtos.java     |  463 ++-
 .../generated/ServerCachingProtos.java          |  221 +-
 .../phoenix/execute/ClientAggregatePlan.java    |   40 +-
 .../apache/phoenix/execute/HashJoinPlan.java    |  104 +-
 .../apache/phoenix/execute/MutationState.java   |    6 +-
 .../PhoenixTxIndexMutationGenerator.java        |    2 +-
 .../phoenix/expression/InListExpression.java    |    5 +-
 .../phoenix/expression/LikeExpression.java      |    2 +-
 .../expression/function/FunctionExpression.java |   10 +-
 .../expression/function/InstrFunction.java      |   78 +-
 .../expression/function/InvertFunction.java     |   19 +-
 .../expression/function/PrefixFunction.java     |    6 +-
 .../expression/function/RTrimFunction.java      |    6 +-
 .../function/RoundDateExpression.java           |   22 +-
 .../function/RoundDecimalExpression.java        |    7 +-
 .../expression/function/SubstrFunction.java     |   20 +
 .../apache/phoenix/index/IndexMaintainer.java   |   14 +-
 .../index/PhoenixIndexFailurePolicy.java        |   75 +-
 .../phoenix/iterate/BaseResultIterators.java    |   12 +-
 .../ClientHashAggregatingResultIterator.java    |  210 ++
 .../apache/phoenix/iterate/ExplainTable.java    |   23 +-
 .../NonAggregateRegionScannerFactory.java       |    5 +-
 .../phoenix/iterate/RegionScannerFactory.java   |    7 +-
 .../phoenix/iterate/TableResultIterator.java    |    6 +-
 .../iterate/TableSnapshotResultIterator.java    |    4 +-
 .../phoenix/jdbc/LoggingPhoenixConnection.java  |   50 +-
 .../jdbc/LoggingPhoenixPreparedStatement.java   |   37 +-
 .../phoenix/jdbc/LoggingPhoenixResultSet.java   |   24 +-
 .../phoenix/jdbc/LoggingPhoenixStatement.java   |   49 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   16 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  540 ++--
 .../apache/phoenix/jdbc/PhoenixMetricsLog.java  |   13 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |    8 +-
 .../apache/phoenix/join/HashCacheClient.java    |   24 +-
 .../apache/phoenix/join/HashCacheFactory.java   |   11 +
 .../java/org/apache/phoenix/log/LogWriter.java  |    2 +-
 .../phoenix/mapreduce/CsvBulkImportUtil.java    |    8 +-
 .../mapreduce/index/IndexScrutinyTool.java      |    2 +-
 .../util/PhoenixConfigurationUtil.java          |    7 +-
 .../phoenix/parse/DropTableStatement.java       |    8 +-
 .../java/org/apache/phoenix/parse/HintNode.java |    9 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |    2 +-
 .../phoenix/query/ConnectionQueryServices.java  |   33 +-
 .../query/ConnectionQueryServicesImpl.java      |  232 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   35 +-
 .../query/DelegateConnectionQueryServices.java  |   19 +-
 .../java/org/apache/phoenix/query/KeyRange.java |   28 +-
 .../apache/phoenix/query/QueryConstants.java    |   29 +-
 .../org/apache/phoenix/query/QueryServices.java |    5 +
 .../phoenix/query/QueryServicesOptions.java     |    5 +
 .../SystemCatalogWALEntryFilter.java            |   45 +-
 .../apache/phoenix/schema/DelegateColumn.java   |   15 +
 .../apache/phoenix/schema/DelegateTable.java    |    8 +-
 .../phoenix/schema/LocalIndexDataColumnRef.java |    2 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  188 +-
 .../phoenix/schema/MetaDataSplitPolicy.java     |   26 +-
 .../java/org/apache/phoenix/schema/PColumn.java |   12 +
 .../org/apache/phoenix/schema/PColumnImpl.java  |  113 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    3 +-
 .../java/org/apache/phoenix/schema/PTable.java  |   20 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  321 ++-
 .../org/apache/phoenix/schema/PTableKey.java    |    4 +-
 .../schema/ParentTableNotFoundException.java    |   47 +
 .../org/apache/phoenix/schema/RowKeySchema.java |   78 +
 .../org/apache/phoenix/schema/SaltingUtil.java  |    4 +-
 .../apache/phoenix/schema/TableProperty.java    |   22 +-
 .../apache/phoenix/schema/types/PVarbinary.java |    4 +-
 .../java/org/apache/phoenix/util/DateUtil.java  |   22 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |   16 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  175 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |    1 -
 .../org/apache/phoenix/util/SchemaUtil.java     |   43 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  186 +-
 .../phoenix/util/csv/CsvUpsertExecutor.java     |    4 +-
 .../phoenix/util/json/JsonUpsertExecutor.java   |    4 +-
 .../apache/phoenix/cache/TenantCacheTest.java   |  112 +-
 .../phoenix/compile/QueryCompilerTest.java      |    4 +-
 .../phoenix/compile/QueryOptimizerTest.java     |    5 +-
 .../TenantSpecificViewIndexCompileTest.java     |   14 +-
 .../phoenix/compile/WhereOptimizerTest.java     |  371 ++-
 .../coprocessor/MetaDataEndpointImplTest.java   |  299 ++
 .../phoenix/execute/CorrelatePlanTest.java      |    5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |    6 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |    7 +-
 .../expression/ColumnExpressionTest.java        |    9 +-
 .../RoundFloorCeilExpressionsTest.java          |   59 +-
 .../expression/function/InstrFunctionTest.java  |   44 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  312 +-
 .../apache/phoenix/query/KeyRangeClipTest.java  |    2 +-
 .../org/apache/phoenix/query/QueryPlanTest.java |    8 +-
 .../apache/phoenix/schema/RowKeySchemaTest.java |   48 +
 .../util/AbstractUpsertExecutorTest.java        |   12 +-
 .../apache/phoenix/util/MetaDataUtilTest.java   |   22 +-
 .../util/TenantIdByteConversionTest.java        |   30 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   23 +-
 phoenix-load-balancer/pom.xml                   |    1 +
 ...apache.phoenix.queryserver.register.Registry |    1 +
 phoenix-protocol/src/main/MetaDataService.proto |   12 +-
 phoenix-protocol/src/main/PTable.proto          |    9 +-
 .../src/main/ServerCachingService.proto         |    2 +
 phoenix-protocol/src/main/build-proto.sh        |    6 +
 .../phoenix/end2end/QueryServerTestUtil.java    |  187 ++
 .../phoenix/end2end/ServerCustomizersIT.java    |  149 +
 .../AvaticaServerConfigurationFactory.java      |   37 +
 .../phoenix/queryserver/server/QueryServer.java |  173 +-
 .../server/ServerCustomizersFactory.java        |   52 +
 .../CustomAvaticaServerConfigurationTest.java   |   37 +
 .../server/QueryServerConfigurationTest.java    |   26 +-
 .../server/ServerCustomizersTest.java           |   92 +
 .../apache/phoenix/tracingwebapp/http/Main.java |   13 +-
 .../tracingwebapp/http/TraceServlet.java        |   87 +-
 pom.xml                                         |   24 +-
 201 files changed, 13225 insertions(+), 5431 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 3a368bc,e97a40d..3cb4f6c
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@@ -33,26 -35,31 +35,36 @@@ import java.sql.ResultSetMetaData
  import java.sql.SQLException;
  import java.util.Arrays;
  import java.util.Collection;
+ import java.util.List;
 +import java.util.Properties;
  
  import org.apache.commons.lang.ArrayUtils;
- import org.apache.hadoop.hbase.client.HTableInterface;
+ import org.apache.hadoop.hbase.client.HTable;
+ import org.apache.hadoop.hbase.client.Result;
+ import org.apache.hadoop.hbase.client.ResultScanner;
+ import org.apache.hadoop.hbase.client.Scan;
+ import org.apache.hadoop.hbase.client.Table;
  import org.apache.hadoop.hbase.util.Bytes;
  import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
  import org.apache.phoenix.exception.SQLExceptionCode;
  import org.apache.phoenix.jdbc.PhoenixConnection;
  import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
  import org.apache.phoenix.query.QueryConstants;
 +import org.apache.phoenix.query.QueryServices;
+ import org.apache.phoenix.schema.ColumnNotFoundException;
+ import org.apache.phoenix.schema.PColumn;
  import org.apache.phoenix.schema.PName;
  import org.apache.phoenix.schema.PNameFactory;
  import org.apache.phoenix.schema.PTable;
  import org.apache.phoenix.schema.PTableKey;
  import org.apache.phoenix.schema.PTableType;
+ import org.apache.phoenix.schema.TableNotFoundException;
 +import org.apache.phoenix.transaction.TransactionFactory;
+ import org.apache.phoenix.util.IndexUtil;
+ import org.apache.phoenix.util.PhoenixRuntime;
 +import org.apache.phoenix.util.PropertiesUtil;
- import org.apache.phoenix.util.StringUtil;
+ import org.apache.phoenix.util.SchemaUtil;
 +import org.apache.phoenix.util.TestUtil;
  import org.junit.Test;
  import org.junit.runner.RunWith;
  import org.junit.runners.Parameterized;
@@@ -756,15 -867,11 +872,16 @@@ public class AlterTableWithViewsIT exte
      
      @Test
      public void testMakeBaseTableTransactional() throws Exception {
 -        try (Connection conn = DriverManager.getConnection(getUrl());
 +        if (!TransactionFactory.Provider.TEPHRA.runTests()) {
 +            return;
 +        }
 +        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
 +        props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
 +        try (Connection conn = DriverManager.getConnection(getUrl(), props);
                  Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {  
-             String baseTableName = "NONTXNTBL_" + generateUniqueName() + (isMultiTenant ? "0":"1");
-             String viewOfTable = baseTableName + "_VIEW";
+             String baseTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+             String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+             
              String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTableName + " ("
                              + " %s ID char(1) NOT NULL,"
                              + " COL1 integer NOT NULL,"

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index c1c033b,5e465b4..227089b
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@@ -47,145 -74,140 +74,141 @@@ import org.apache.phoenix.jdbc.PhoenixS
  import org.apache.phoenix.query.KeyRange;
  import org.apache.phoenix.query.QueryServices;
  import org.apache.phoenix.schema.ColumnAlreadyExistsException;
+ import org.apache.phoenix.schema.ConcurrentTableMutationException;
+ import org.apache.phoenix.schema.PColumn;
+ import org.apache.phoenix.schema.PTable;
+ import org.apache.phoenix.schema.PTableType;
  import org.apache.phoenix.schema.ReadOnlyTableException;
  import org.apache.phoenix.schema.TableNotFoundException;
+ import org.apache.phoenix.util.MetaDataUtil;
  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.SchemaUtil;
+ import org.apache.phoenix.util.TestUtil;
+ import org.junit.BeforeClass;
  import org.junit.Test;
- 
- 
- 
- public class ViewIT extends BaseViewIT {
- 	
-     public ViewIT(String txProvider) {
- 		super(txProvider);
- 	}
- 
-     @Test
-     public void testReadOnlyOnReadOnlyView() throws Exception {
-         Connection earlierCon = DriverManager.getConnection(getUrl());
-         Connection conn = DriverManager.getConnection(getUrl());
-         String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) "+ tableDDLOptions;
-         conn.createStatement().execute(ddl);
-         String fullParentViewName = "V_" + generateUniqueName();
-         ddl = "CREATE VIEW " + fullParentViewName + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
-         conn.createStatement().execute(ddl);
-         try {
-             conn.createStatement().execute("UPSERT INTO " + fullParentViewName + " VALUES(1)");
-             fail();
-         } catch (ReadOnlyTableException e) {
-             
-         }
-         for (int i = 0; i < 10; i++) {
-             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ")");
-         }
-         conn.commit();
-         
-         analyzeTable(conn, fullParentViewName, txProvider != null);
-         
-         List<KeyRange> splits = getAllSplits(conn, fullParentViewName);
-         assertEquals(4, splits.size());
-         
-         int count = 0;
-         ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
-         while (rs.next()) {
-             assertEquals(count++, rs.getInt(1));
-         }
-         assertEquals(10, count);
-         
-         count = 0;
-         rs = conn.createStatement().executeQuery("SELECT k FROM " + fullParentViewName);
-         while (rs.next()) {
-             count++;
-             assertEquals(count + 5, rs.getInt(1));
-         }
-         assertEquals(4, count);
-         count = 0;
-         rs = earlierCon.createStatement().executeQuery("SELECT k FROM " + fullParentViewName);
-         while (rs.next()) {
-             count++;
-             assertEquals(count + 5, rs.getInt(1));
-         }
-         assertEquals(4, count);
-         String fullViewName = "V_" + generateUniqueName();
-         ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullParentViewName + " WHERE k < 9";
-         conn.createStatement().execute(ddl);
-         try {
-             conn.createStatement().execute("UPSERT INTO " + fullViewName + " VALUES(1)");
-             fail();
-         } catch (ReadOnlyTableException e) {
-             
-         } finally {
-             conn.close();
+ import org.junit.runner.RunWith;
+ import org.junit.runners.Parameterized;
+ import org.junit.runners.Parameterized.Parameters;
+ 
+ import com.google.common.base.Predicate;
+ import com.google.common.collect.Collections2;
+ import com.google.common.collect.Maps;
+ @RunWith(Parameterized.class)
+ public class ViewIT extends SplitSystemCatalogIT {
+ 
+     protected String tableDDLOptions;
 -    protected boolean transactional;
++    protected String transactionProvider;
+     protected boolean columnEncoded;
+     
+     private static final String FAILED_VIEWNAME = SchemaUtil.getTableName(SCHEMA2, "FAILED_VIEW");
+     private static final String SLOW_VIEWNAME_PREFIX = SchemaUtil.getTableName(SCHEMA2, "SLOW_VIEW");
+ 
+     private static volatile CountDownLatch latch1 = null;
+     private static volatile CountDownLatch latch2 = null;
+ 
 -    public ViewIT(boolean transactional, boolean columnEncoded) {
++    public ViewIT(String transactionProvider, boolean columnEncoded) {
+         StringBuilder optionBuilder = new StringBuilder();
 -        this.transactional = transactional;
++        this.transactionProvider = transactionProvider;
+         this.columnEncoded = columnEncoded;
 -        if (transactional) {
 -            optionBuilder.append(" TRANSACTIONAL=true ");
++        if (transactionProvider != null) {
++            optionBuilder.append(" TRANSACTION_PROVIDER='" + transactionProvider + "'");
          }
- 
-         conn = DriverManager.getConnection(getUrl());
-         count = 0;
-         rs = conn.createStatement().executeQuery("SELECT k FROM " + fullViewName);
-         while (rs.next()) {
-             count++;
-             assertEquals(count + 5, rs.getInt(1));
+         if (!columnEncoded) {
+             if (optionBuilder.length()!=0)
+                 optionBuilder.append(",");
+             optionBuilder.append("COLUMN_ENCODED_BYTES=0");
          }
-         assertEquals(3, count);
+         this.tableDDLOptions = optionBuilder.toString();
      }
  
-     @Test
-     public void testNonSaltedUpdatableViewWithIndex() throws Exception {
-         testUpdatableViewWithIndex(null, false);
 -    @Parameters(name="ViewIT_transactional={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
 -    public static Collection<Boolean[]> data() {
 -        return Arrays.asList(new Boolean[][] { 
 -            { true, false }, { true, true },
 -            { false, false }, { false, true }});
++    @Parameters(name="ViewIT_transactionProvider={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
++    public static Collection<Object[]> data() {
++        return Arrays.asList(new Object[][] { 
++            { "TEPHRA", false }, { "TEPHRA", true },
++            { "OMID", false }, 
++            { null, false }, { null, true }});
      }
      
-     @Test
-     public void testNonSaltedUpdatableViewWithLocalIndex() throws Exception {
-         testUpdatableViewWithIndex(null, true);
+     @BeforeClass
+     public static void doSetup() throws Exception {
+         NUM_SLAVES_BASE = 6;
+         Map<String, String> props = Collections.emptyMap();
+         boolean splitSystemCatalog = (driver == null);
+         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+         serverProps.put(QueryServices.PHOENIX_ACLS_ENABLED, "true");
+         serverProps.put(PhoenixMetaDataCoprocessorHost.PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
+             TestMetaDataRegionObserver.class.getName());
+         serverProps.put("hbase.coprocessor.abortonerror", "false");
+         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(props.entrySet().iterator()));
+         // Split SYSTEM.CATALOG once after the mini-cluster is started
+         if (splitSystemCatalog) {
+             splitSystemCatalog();
+         }
      }
      
-     @Test
-     public void testUpdatableOnUpdatableView() throws Exception {
-         String viewName = testUpdatableView(null);
-         Connection conn = DriverManager.getConnection(getUrl());
-         String fullViewName = "V_" + generateUniqueName();
-         String ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + viewName + " WHERE k3 = 2";
-         conn.createStatement().execute(ddl);
-         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName);
-         assertTrue(rs.next());
-         assertEquals(1, rs.getInt(1));
-         assertEquals(109, rs.getInt(2));
-         assertEquals(2, rs.getInt(3));
-         assertFalse(rs.next());
- 
-         conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2) VALUES(122)");
-         conn.commit();
-         rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName + " WHERE k2 >= 120");
-         assertTrue(rs.next());
-         assertEquals(1, rs.getInt(1));
-         assertEquals(122, rs.getInt(2));
-         assertEquals(2, rs.getInt(3));
-         assertFalse(rs.next());
+     public static class TestMetaDataRegionObserver extends BaseMetaDataEndpointObserver {
          
-         try {
-             conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2,k3) VALUES(123,3)");
-             fail();
-         } catch (SQLException e) {
-             assertEquals(SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN.getErrorCode(), e.getErrorCode());
+         @Override
+         public void preAlterTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+                 String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType type) throws IOException{
+             processTable(tableName);
+         }
+         
+         @Override
+         public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+                 String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+                 Set<byte[]> familySet, Set<TableName> indexes) throws IOException {
+             processTable(tableName);
          }
  
-         try {
-             conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2,k3) select k2, 3 from " + fullViewName);
-             fail();
-         } catch (SQLException e) {
-             assertEquals(SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN.getErrorCode(), e.getErrorCode());
+         @Override
+         public void preDropTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+                 String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+                 List<PTable> indexes) throws IOException {
+             processTable(tableName);
          }
-     }
  
+         private void processTable(String tableName) throws DoNotRetryIOException {
+             if (tableName.equals(FAILED_VIEWNAME)) {
+                 // throwing anything other than instances of IOException result
+                 // in this coprocessor being unloaded
+                 // DoNotRetryIOException tells HBase not to retry this mutation
+                 // multiple times
+                 throw new DoNotRetryIOException();
+             } else if (tableName.startsWith(SLOW_VIEWNAME_PREFIX)) {
+                 // simulate a slow write to SYSTEM.CATALOG
+                 if (latch1 != null) {
+                     latch1.countDown();
+                 }
+                 if (latch2 != null) {
+                     try {
+                         // wait till the second task is complete before completing the first task
+                         boolean result = latch2.await(2, TimeUnit.MINUTES);
+                         if (!result) {
+                             throw new RuntimeException("Second task took took long to complete");
+                         }
+                     } catch (InterruptedException e) {
+                     }
+                 }
+             }
+         }
+         
+     }
+     
      @Test
      public void testReadOnlyOnUpdatableView() throws Exception {
-         String viewName = testUpdatableView(null);
+         String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+         String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+         String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+         String ddl =
+                 "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1
+                         + " WHERE k3 > 1 and k3 < 50";
+         testUpdatableView(fullTableName, fullViewName1, fullViewName2, ddl, null, tableDDLOptions);
          Connection conn = DriverManager.getConnection(getUrl());
-         String fullViewName = "V_" + generateUniqueName();
-         String ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + viewName + " WHERE k3 > 1 and k3 < 50";
-         conn.createStatement().execute(ddl);
-         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName);
+         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2);
          assertTrue(rs.next());
          assertEquals(1, rs.getInt(1));
          assertEquals(109, rs.getInt(2));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 7efe17e,0611478..f09901e
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@@ -28,8 -28,8 +28,9 @@@ import java.sql.Connection
  import java.sql.PreparedStatement;
  import java.sql.ResultSet;
  import java.sql.SQLException;
+ import java.util.ArrayList;
  import java.util.Arrays;
 +import java.util.Collection;
  import java.util.List;
  import java.util.Map;
  import java.util.Properties;
@@@ -105,15 -110,15 +111,15 @@@ public class MutableIndexFailureIT exte
      private final boolean throwIndexWriteFailure;
      private String schema = generateUniqueName();
      private List<CommitException> exceptions = Lists.newArrayList();
-     private static RegionCoprocessorEnvironment indexRebuildTaskRegionEnvironment;
-     private static final int forwardOverlapMs = 1000;
-     private static final int disableTimestampThresholdMs = 10000;
-     private static final int numRpcRetries = 2;
+     protected static RegionCoprocessorEnvironment indexRebuildTaskRegionEnvironment;
+     protected static final int forwardOverlapMs = 1000;
+     protected static final int disableTimestampThresholdMs = 10000;
+     protected static final int numRpcRetries = 2;
  
 -    public MutableIndexFailureIT(boolean transactional, boolean localIndex, boolean isNamespaceMapped, Boolean disableIndexOnWriteFailure, boolean failRebuildTask, Boolean throwIndexWriteFailure) {
 -        this.transactional = transactional;
 +    public MutableIndexFailureIT(String transactionProvider, boolean localIndex, boolean isNamespaceMapped, Boolean disableIndexOnWriteFailure, boolean failRebuildTask, Boolean throwIndexWriteFailure) {
 +        this.transactional = transactionProvider != null;
          this.localIndex = localIndex;
 -        this.tableDDLOptions = " SALT_BUCKETS=2, COLUMN_ENCODED_BYTES=NONE" + (transactional ? ", TRANSACTIONAL=true " : "") 
 +        this.tableDDLOptions = " SALT_BUCKETS=2, COLUMN_ENCODED_BYTES=NONE" + (transactional ? (",TRANSACTIONAL=true,TRANSACTION_PROVIDER='"+transactionProvider+"'") : "") 
                  + (disableIndexOnWriteFailure == null ? "" : (", " + PhoenixIndexFailurePolicy.DISABLE_INDEX_ON_WRITE_FAILURE + "=" + disableIndexOnWriteFailure))
                  + (throwIndexWriteFailure == null ? "" : (", " + PhoenixIndexFailurePolicy.THROW_INDEX_WRITE_FAILURE + "=" + throwIndexWriteFailure));
          this.tableName = FailingRegionObserver.FAIL_TABLE_NAME;
@@@ -143,53 -165,29 +166,41 @@@
           * because we want to control it's execution ourselves
           */
          serverProps.put(QueryServices.INDEX_REBUILD_TASK_INITIAL_DELAY, Long.toString(Long.MAX_VALUE));
-         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
-         clientProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
-         NUM_SLAVES_BASE = 4;
-         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
-         indexRebuildTaskRegionEnvironment =
-                 (RegionCoprocessorEnvironment) getUtility()
-                         .getRSForFirstRegionInTable(
-                             PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
-                         .getOnlineRegions(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
-                         .get(0).getCoprocessorHost()
-                         .findCoprocessorEnvironment(MetaDataRegionObserver.class.getName());
-         MetaDataRegionObserver.initRebuildIndexConnectionProps(
-             indexRebuildTaskRegionEnvironment.getConfiguration());
+         return serverProps;
      }
  
 -    @Parameters(name = "MutableIndexFailureIT_transactional={0},localIndex={1},isNamespaceMapped={2},disableIndexOnWriteFailure={3},failRebuildTask={4},throwIndexWriteFailure={5}") // name is used by failsafe as file name in reports
 -    public static List<Object[]> data() {
 -        return Arrays.asList(new Object[][] { 
 -                // note - can't disableIndexOnWriteFailure without throwIndexWriteFailure, PHOENIX-4130
 -                { false, false, false, false, false, false},
 -                { false, false, false, true, false, null},
 -                { true, false, false, true, false, null},
 -                { false, true, false, null, false, null},
 -                { true, true, false, true, false, null},
 -                { false, false, false, false, false, null},
 -                { false, true, false, false, false, null},
 -                { false, false, false, false, false, null},
 -                { false, false, false, true, false, null},
 -                { false, false, false, true, false, null},
 -                { false, true, false, true, false, null},
 -                { false, true, false, true, false, null},
 -                { false, false, false, true, true, null},
 -                { false, false, false, false, true, false},
 -                } 
 -        );
 +    @Parameters(name = "MutableIndexFailureIT_transactionProvider={0},localIndex={1},isNamespaceMapped={2},disableIndexOnWriteFailure={3},failRebuildTask={4},throwIndexWriteFailure={5}") // name is used by failsafe as file name in reports
 +    public static Collection<Object[]> data() {
 +        return TestUtil.filterTxParamData(
 +                Arrays.asList(new Object[][] { 
 +                    // note - can't disableIndexOnWriteFailure without throwIndexWriteFailure, PHOENIX-4130
 +                    { null, false, false, false, false, false},
 +                    { null, false, true, true, false, null},
 +                    { null, false, true, true, false, true},
 +                    { null, false, false, true, false, null},
 +                    { null, true, true, true, false, null},
 +                    { null, true, false, null, false, null},
 +                    { "TEPHRA", true, false, false, false, null},
 +                    { "TEPHRA", true, true, false, false, null},
 +                    { "TEPHRA", false, false, false, false, null},
 +                    { "TEPHRA", false, true, false, false, null},
 +                    { "OMID", true, false, false, false, null},
 +                    { "OMID", true, true, false, false, null},
 +                    { "OMID", false, false, false, false, null},
 +                    { "OMID", false, true, false, false, null},
 +    
 +                    { null, false, false, false, false, null},
 +                    { null, true, false, false, false, null},
 +                    { null, false, false, false, false, null},
 +                    { null, false, false, true, false, null},
 +                    { null, false, false, true, false, null},
 +                    { null, true, false, true, false, null},
 +                    { null, true, false, true, false, null},
 +                    { null, false, false, true, true, null},
 +                    { null, false, true, true, true, null},
 +                    { null, false, false, false, true, false},
 +                    { null, false, true, false, true, false},
 +                    }), 0);
      }
  
      private void runRebuildTask(Connection conn) throws InterruptedException, SQLException {
@@@ -409,12 -412,62 +425,62 @@@
          stmt.setString(3, "3");
          stmt.execute();
          conn.commit();
+     }
  
+     private void addRowsInTableDuringRetry(final String tableName)
+             throws SQLException, InterruptedException, ExecutionException {
+         int threads=10;
+         boolean wasFailWrite = FailingRegionObserver.FAIL_WRITE;
+         boolean wasToggleFailWriteForRetry = FailingRegionObserver.TOGGLE_FAIL_WRITE_FOR_RETRY;
+         try {
+             Callable callable = new Callable() {
+ 
+                 @Override
+                 public Boolean call() {
+                     Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+                     props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, String.valueOf(isNamespaceMapped));
+                     try (Connection conn = driver.connect(url, props)) {
+                         // In case of disable index on failure policy, INDEX will be in PENDING_DISABLE on first retry
+                         // but will
+                         // become active if retry is successfull
+                         PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
+                         stmt.setString(1, "b");
+                         stmt.setString(2, "y");
+                         stmt.setString(3, "2");
+                         stmt.execute();
+                         if (!leaveIndexActiveOnFailure && !transactional) {
+                             FailingRegionObserver.FAIL_WRITE = true;
+                             FailingRegionObserver.TOGGLE_FAIL_WRITE_FOR_RETRY = true;
+                         }
+                         conn.commit();
+                     } catch (SQLException e) {
+                         return false;
+                     }
+                     return true;
+                 }
+             };
+             ExecutorService executor = Executors.newFixedThreadPool(threads);
+             List<Future<Boolean>> futures = new ArrayList<Future<Boolean>>();
+             for (int i = 0; i < threads; i++) {
+                 futures.add(executor.submit(callable));
+             }
+             for (Future<Boolean> future : futures) {
+                 Boolean isSuccess = future.get();
+                 // transactions can have conflict so ignoring the check for them
+                 if (!transactional) {
+                     assertTrue(isSuccess);
+                 }
+             }
+             executor.shutdown();
+         } finally {
+             FailingRegionObserver.FAIL_WRITE = wasFailWrite;
+             FailingRegionObserver.TOGGLE_FAIL_WRITE_FOR_RETRY = wasToggleFailWriteForRetry;
+         }
      }
  
 -    private void validateDataWithIndex(Connection conn, String fullTableName, String fullIndexName, boolean localIndex) throws SQLException {
 +    private void validateDataWithIndex(Connection conn, String fullTableName, String fullIndexName, boolean localIndex) throws Exception {
          String query = "SELECT /*+ INDEX(" + fullTableName + " " + SchemaUtil.getTableNameFromFullName(fullIndexName) + ")  */ k,v1 FROM " + fullTableName;
 -        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 +        ResultSet rs = conn.createStatement().executeQuery(query);
          String expectedPlan = " OVER "
                  + (localIndex
                          ? Bytes.toString(

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureWithNamespaceIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureWithNamespaceIT.java
index 0000000,5ed9e1f..d5f7bd1
mode 000000,100644..100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureWithNamespaceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureWithNamespaceIT.java
@@@ -1,0 -1,80 +1,82 @@@
+ /*
+  * 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.index;
+ 
+ import java.util.Arrays;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import org.apache.hadoop.hbase.HConstants;
+ import org.apache.hadoop.hbase.TableName;
+ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+ import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
+ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+ import org.apache.phoenix.query.QueryServices;
+ import org.apache.phoenix.util.ReadOnlyProps;
+ import org.apache.phoenix.util.SchemaUtil;
+ import org.junit.BeforeClass;
+ import org.junit.runners.Parameterized.Parameters;
+ 
+ import com.google.common.collect.Maps;
+ 
+ /*
+  * This class is to ensure gets its own cluster with Namespace Enabled
+  */
+ public class MutableIndexFailureWithNamespaceIT extends MutableIndexFailureIT {
+ 
 -    public MutableIndexFailureWithNamespaceIT(boolean transactional, boolean localIndex, boolean isNamespaceMapped,
++    public MutableIndexFailureWithNamespaceIT(String transactionProvider, boolean localIndex, boolean isNamespaceMapped,
+             Boolean disableIndexOnWriteFailure, boolean failRebuildTask, Boolean throwIndexWriteFailure) {
 -        super(transactional, localIndex, isNamespaceMapped, disableIndexOnWriteFailure, failRebuildTask,
++        super(transactionProvider, localIndex, isNamespaceMapped, disableIndexOnWriteFailure, failRebuildTask,
+                 throwIndexWriteFailure);
+     }
+     
+     @BeforeClass
+     public static void doSetup() throws Exception {
+         Map<String, String> serverProps = getServerProps();
+         serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());
+         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+         clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());
+         clientProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+         NUM_SLAVES_BASE = 4;
+         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+         TableName systemTable = SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
+                 true);
+         indexRebuildTaskRegionEnvironment = (RegionCoprocessorEnvironment)getUtility()
+                 .getRSForFirstRegionInTable(systemTable).getOnlineRegions(systemTable).get(0).getCoprocessorHost()
+                 .findCoprocessorEnvironment(MetaDataRegionObserver.class.getName());
+         MetaDataRegionObserver.initRebuildIndexConnectionProps(indexRebuildTaskRegionEnvironment.getConfiguration());
+     }
+     
+     @Parameters(name = "MutableIndexFailureIT_transactional={0},localIndex={1},isNamespaceMapped={2},disableIndexOnWriteFailure={3},failRebuildTask={4},throwIndexWriteFailure={5}") // name is used by failsafe as file name in reports
+     public static List<Object[]> data() {
+         return Arrays.asList(new Object[][] { 
+                 // note - can't disableIndexOnWriteFailure without throwIndexWriteFailure, PHOENIX-4130
 -                { false, false, true, true, false, null},
 -                { false, false, true, true, false, true},
 -                { true, false, true, true, false, null},
 -                { false, true, true, true, false, null},
 -                { true, true, true, null, false, null},
 -                { false, false, true, true, true, null},
 -                { false, false, true, false, true, false},
++                { null, false, true, true, false, null},
++                { null, false, true, true, false, true},
++                { "TEPHRA", false, true, true, false, null},
++                { "OMID", false, true, true, false, null},
++                { null, true, true, true, false, null},
++                { "TEPHRA", true, true, null, false, null},
++                { "OMID", true, true, null, false, null},
++                { null, false, true, true, true, null},
++                { null, false, true, false, true, false},
+                 } 
+         );
+     }
+ 
+ }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 85afd66,4433e12..2a58a6e
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@@ -106,16 -106,16 +106,16 @@@ public class MutableIndexIT extends Par
          return getConnection(props);
      }
      
- 	@Parameters(name="MutableIndexIT_localIndex={0},transactionProvider={1},columnEncoded={2}") // name is used by failsafe as file name in reports
 -    @Parameters(name="MutableIndexIT_localIndex={0},transactional={1},columnEncoded={2}") // name is used by failsafe as file name in reports
++    @Parameters(name="MutableIndexIT_localIndex={0},transactionProvider={1},columnEncoded={2}") // name is used by failsafe as file name in reports
      public static Collection<Object[]> data() {
 -        return Arrays.asList(new Object[][] { 
 +        return TestUtil.filterTxParamData(Arrays.asList(new Object[][] { 
                  { false, null, false }, { false, null, true },
                  { false, "TEPHRA", false }, { false, "TEPHRA", true },
 -                //{ false, "OMID", false }, { false, "OMID", true },
 +                { false, "OMID", false },
                  { true, null, false }, { true, null, true },
                  { true, "TEPHRA", false }, { true, "TEPHRA", true },
 -                //{ true, "OMID", false }, { true, "OMID", true },
 -                });
 +                { true, "OMID", false },
 +                }),1);
      }
      
      @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index 6a5c73d,feb0ce4..c6dc312
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@@ -264,16 -284,16 +264,16 @@@ public class PartialCommitIT extends Ba
      }
      
      private PhoenixConnection getConnectionWithTableOrderPreservingMutationState() throws SQLException {
 -        Connection con = driver.connect(url, new Properties());
 -        PhoenixConnection phxCon = new PhoenixConnection(con.unwrap(PhoenixConnection.class));
 -        final Map<TableRef, MultiRowMutationState> mutations = Maps.newTreeMap(new TableRefComparator());
 -        // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
 -        return new PhoenixConnection(phxCon, (MutationState)null) {
 -            @Override
 -            protected MutationState newMutationState(int maxSize, int maxSizeBytes) {
 -                return new MutationState(maxSize, maxSizeBytes, this, mutations, false, null);
 +        try (PhoenixConnection con = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class)) {
 +            final Map<TableRef, MultiRowMutationState> mutations = Maps.newTreeMap(new TableRefComparator());
 +            // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
-             return new PhoenixConnection(con, null) {
++            return new PhoenixConnection(con, (MutationState)null) {
 +                @Override
 +                protected MutationState newMutationState(int maxSize, int maxSizeBytes) {
 +                    return new MutationState(maxSize, maxSizeBytes, this, mutations, false, null);
 +                };
              };
 -        };
 +        }
      }
  
      public static class FailingRegionObserver extends SimpleRegionObserver {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index 009ee06,2211d58..0ddbed3
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@@ -40,8 -41,8 +40,9 @@@ import org.apache.phoenix.query.Connect
  import org.apache.phoenix.query.QueryConstants;
  import org.apache.phoenix.schema.MetaDataClient;
  import org.apache.phoenix.schema.PName;
+ import org.apache.phoenix.schema.PTable;
  import org.apache.phoenix.schema.types.PVarchar;
 +import org.apache.phoenix.transaction.TransactionFactory;
  import org.apache.phoenix.util.PropertiesUtil;
  import org.apache.phoenix.util.SchemaUtil;
  import org.apache.phoenix.util.TestUtil;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 05cf21a,12c3b7a..1a8cb19
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@@ -43,12 -40,8 +44,13 @@@ import org.apache.hadoop.hbase.HBaseIOE
  import org.apache.hadoop.hbase.HColumnDescriptor;
  import org.apache.hadoop.hbase.HTableDescriptor;
  import org.apache.hadoop.hbase.TableName;
+ import org.apache.hadoop.hbase.TableNotFoundException;
  import org.apache.hadoop.hbase.client.Admin;
 +import org.apache.hadoop.hbase.client.Mutation;
 +import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 +import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 +import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
  import org.apache.hadoop.hbase.util.Bytes;
  import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
  import org.apache.phoenix.exception.SQLExceptionCode;
@@@ -473,8 -376,7 +475,8 @@@ public class TransactionIT  extends Par
          }
      }
      
 +    
-     private static void assertTTL(Admin admin, String tableName, int ttl) throws Exception {
+     private static void assertTTL(Admin admin, String tableName, int ttl) throws TableNotFoundException, IOException {
          HTableDescriptor tableDesc = admin.getTableDescriptor(TableName.valueOf(tableName));
          for (HColumnDescriptor colDesc : tableDesc.getFamilies()) {
              assertEquals(ttl,Integer.parseInt(colDesc.getValue(TxConstants.PROPERTY_TTL)));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 70b9d34,c2dfeab..583085e
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@@ -975,9 -974,7 +975,9 @@@ public class DeleteCompiler 
      private static boolean isMaintainedOnClient(PTable table) {
          // Test for not being local (rather than being GLOBAL) so that this doesn't fail
          // when tested with our projected table.
 -        return table.getIndexType() != IndexType.LOCAL && (table.isImmutableRows() || table.isTransactional());
 +        return (table.getIndexType() != IndexType.LOCAL && (table.isTransactional() || table.isImmutableRows())) ||
 +               (table.getIndexType() == IndexType.LOCAL && (table.isTransactional() &&
 +                table.getTransactionProvider().getTransactionProvider().isUnsupported(Feature.MAINTAIN_LOCAL_INDEX_ON_SERVER) ) );
      }
      
- }
+ }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
index 6ef6f3b,877c939..1737911
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
@@@ -191,9 -177,9 +191,9 @@@ public class PhoenixTxIndexMutationGene
              
              // Project empty key value column
              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 scanRanges = ScanRanges.create(SchemaUtil.VAR_BINARY_SCHEMA, Collections.singletonList(keys), ScanUtil.SINGLE_COLUMN_SLOT_SPAN, null, true, -1);
              scanRanges.initializeScan(scan);
 -            Table txTable = indexMetaData.getTransactionContext().getTransactionalTable(htable, isImmutable);
 +            Table txTable = indexMetaData.getTransactionContext().getTransactionalTable(htable, true);
              // For rollback, we need to see all versions, including
              // the last committed version as there may be multiple
              // checkpointed versions.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index eb79fb9,0820232..d5a5199
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@@ -155,5 -168,19 +168,19 @@@ public interface ConnectionQueryService
  
      public QueryLoggerDisruptor getQueryDisruptor();
      
 -    public PhoenixTransactionClient initTransactionClient(TransactionFactory.Provider provider);
 +    public PhoenixTransactionClient initTransactionClient(TransactionFactory.Provider provider) throws SQLException;
- }
+     
+     /**
+      * Writes a cell to SYSTEM.MUTEX using checkAndPut to ensure only a single client can execute a
+      * particular task. The params are used to generate the rowkey.
+      * @return true if this client was able to successfully acquire the mutex
+      */
+     public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
+             String columnName, String familyName) throws SQLException;
+ 
+     /**
+      * Deletes a cell that was written to SYSTEM.MUTEX. The params are used to generate the rowkey.
+      */
+     public void deleteMutexCell(String tenantId, String schemaName, String tableName,
+             String columnName, String familyName) throws SQLException;
 -}
++}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 262da3c,147e873..4be4af8
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@@ -367,7 -367,18 +367,18 @@@ public class DelegateConnectionQuerySer
      }
      
      @Override
 -    public PhoenixTransactionClient initTransactionClient(Provider provider) {
 +    public PhoenixTransactionClient initTransactionClient(Provider provider) throws SQLException {
          return getDelegate().initTransactionClient(provider);
      }
+ 
+     @Override
+     public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
+             String columnName, String familyName) throws SQLException {
+         return true;
+     }
+ 
+     @Override
+     public void deleteMutexCell(String tenantId, String schemaName, String tableName,
+             String columnName, String familyName) throws SQLException {
+     }
  }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/24178207/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 14dd94f,4412c15..8e1f05c
--- a/pom.xml
+++ b/pom.xml
@@@ -98,10 -98,9 +98,10 @@@
      <!-- Do not change jodatime.version until HBASE-15199 is fixed -->
      <jodatime.version>1.6</jodatime.version>
      <joni.version>2.1.2</joni.version>
-     <avatica.version>1.10.0</avatica.version>
+     <avatica.version>1.12.0</avatica.version>
      <jettyVersion>8.1.7.v20120910</jettyVersion>
      <tephra.version>0.14.0-incubating</tephra.version>
 +    <omid.version>0.8.2.11-SNAPSHOT</omid.version>
      <spark.version>2.0.2</spark.version>
      <scala.version>2.11.8</scala.version>
      <scala.binary.version>2.11</scala.binary.version>
@@@ -142,8 -141,8 +142,8 @@@
            <artifactId>maven-compiler-plugin</artifactId>
            <version>3.0</version>
            <configuration>
--            <source>1.7</source>
--            <target>1.7</target>
++            <source>1.8</source>
++            <target>1.8</target>
            </configuration>
          </plugin>
          <!--This plugin's configuration is used to store Eclipse m2e settings 


[10/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 ab3a4ab..e39d492 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
@@ -21,6 +21,8 @@ import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -33,37 +35,46 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Properties;
+import java.util.List;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.StringUtil;
-import org.apache.phoenix.util.TestUtil;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+
 @RunWith(Parameterized.class)
-public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
-    
+public class AlterTableWithViewsIT extends SplitSystemCatalogIT {
+
     private final boolean isMultiTenant;
     private final boolean columnEncoded;
-    
-    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
-    private final String TENANT_SPECIFIC_URL2 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant2";
+    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
+    private final String TENANT_SPECIFIC_URL2 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT2;
     
     public AlterTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
         this.isMultiTenant = isMultiTenant;
@@ -77,6 +88,14 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                 { true, false }, { true, true } });
     }
     
+    // transform PColumn to String
+    private Function<PColumn,String> function = new Function<PColumn,String>(){
+        @Override
+        public String apply(PColumn input) {
+            return input.getName().getString();
+        }
+    };
+    
     private String generateDDL(String format) {
         return generateDDL("", format);
     }
@@ -101,8 +120,9 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public void testAddNewColumnsToBaseTableWithViews() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {       
-            String tableName = generateUniqueName();
-            String viewOfTable = tableName + "_VIEW";
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL,"
@@ -113,12 +133,13 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // adding a new pk column and a new regular column
             conn.createStatement().execute("ALTER TABLE " + tableName + " ADD COL3 varchar(10) PRIMARY KEY, COL4 integer");
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 7, 5, "ID", "COL1", "COL2", "COL3", "COL4", "VIEW_COL1", "VIEW_COL2");
+            // add/drop column to a base table are no longer propagated to child views
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
         } 
     }
     
@@ -126,9 +147,10 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public void testAlterPropertiesOfParentTable() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {       
-            String tableName = generateUniqueName();
-            String viewOfTable1 = tableName + "_VIEW1";
-            String viewOfTable2 = tableName + "_VIEW2";
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String viewOfTable2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL,"
@@ -138,11 +160,12 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute(generateDDL("UPDATE_CACHE_FREQUENCY=2", ddlFormat));
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
+            
             viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 1");
             
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             PTable table = phoenixConn.getTable(new PTableKey(null, tableName));
-            PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
             assertFalse(table.isImmutableRows());
             assertEquals(2, table.getUpdateCacheFrequency());
             PTable viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
@@ -166,7 +189,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
             assertTrue(viewTable1.isImmutableRows());
-            assertEquals(3, viewTable1.getUpdateCacheFrequency());
+            assertEquals(2, viewTable1.getUpdateCacheFrequency());
             
             viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
             assertTrue(viewTable2.isImmutableRows());
@@ -178,16 +201,20 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             ResultSet rs;
             DatabaseMetaData md = conn.getMetaData();
-            rs = md.getTables("", "", StringUtil.escapeLike(tableName), null);
+            rs =
+                    md.getTables("", SchemaUtil.getSchemaNameFromFullName(tableName),
+                        SchemaUtil.getTableNameFromFullName(tableName), null);
             assertTrue(rs.next());
             assertEquals(gpw, rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH));
             
-            rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), null);
+            rs = md.getTables(null, SchemaUtil.getSchemaNameFromFullName(viewOfTable1),
+                SchemaUtil.getTableNameFromFullName(viewOfTable1), null);
             assertTrue(rs.next());
             rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
             assertTrue(rs.wasNull());
 
-            rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), null);
+            rs = md.getTables(null, SchemaUtil.getSchemaNameFromFullName(viewOfTable2),
+                SchemaUtil.getTableNameFromFullName(viewOfTable2), null);
             assertTrue(rs.next());
             rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
             assertTrue(rs.wasNull());
@@ -198,8 +225,9 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public void testDropColumnsFromBaseTableWithView() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
-            String tableName = generateUniqueName();
-            String viewOfTable = tableName + "_VIEW";
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " (" + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL," + " COL2 bigint NOT NULL,"
                             + " COL3 varchar(10)," + " COL4 varchar(10)," + " COL5 varchar(10),"
@@ -212,15 +240,16 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             viewConn.createStatement()
                     .execute(
                         "CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 8, 6,
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 8, 6,
                 "ID", "COL1", "COL2", "COL3", "COL4", "COL5", "VIEW_COL1", "VIEW_COL2");
 
             // drop two columns from the base table
             conn.createStatement().execute("ALTER TABLE " + tableName + " DROP COLUMN COL3, COL5");
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 4,
                 QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL4");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 6, 4,
-                "ID", "COL1", "COL2", "COL4", "VIEW_COL1", "VIEW_COL2");
+            // the columns will still exist in the view metadata , but are excluded while combining parent table columns
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 8, 6,
+                "ID", "COL1", "COL2", "COL3", "COL4", "COL5", "VIEW_COL1", "VIEW_COL2");
         }
     }
     
@@ -230,33 +259,35 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
-            String tableName = generateUniqueName();
-            String viewOfTable = tableName + "_VIEW";
-
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(10) NOT NULL,"
                             + " COL1 integer NOT NULL,"
                             + " COL2 bigint NOT NULL,"
+                            + " COL3 varchar,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
                             + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 4, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256), VIEW_COL3 VARCHAR, VIEW_COL4 DECIMAL, VIEW_COL5 DECIMAL(10,2), VIEW_COL6 VARCHAR, CONSTRAINT pk PRIMARY KEY (VIEW_COL5, VIEW_COL6) ) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn,viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+            assertTableDefinition(viewConn,viewOfTable, PTableType.VIEW, tableName, 0, 10, 4, "ID", "COL1", "COL2", "COL3", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
             
             // upsert single row into view
-            String dml = "UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?, ?, ?, ?, ?)";
+            String dml = "UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = viewConn.prepareStatement(dml);
             stmt.setString(1, "view1");
             stmt.setInt(2, 12);
             stmt.setInt(3, 13);
-            stmt.setInt(4, 14);
-            stmt.setString(5, "view5");
+            stmt.setString(4, "view4");
+            stmt.setInt(5, 15);
             stmt.setString(6, "view6");
-            stmt.setInt(7, 17);
+            stmt.setString(7, "view7");
             stmt.setInt(8, 18);
-            stmt.setString(9, "view9");
+            stmt.setInt(9, 19);
+            stmt.setString(10, "view10");
             stmt.execute();
             viewConn.commit();
             
@@ -297,8 +328,8 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             }
             
             // validate that there were no columns added to the table or view, if its table is column encoded the sequence number changes when we increment the cq counter
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 1 : 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 1 : 0, 4, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3");
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 10, 4, "ID", "COL1", "COL2", "COL3", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
             
             if (columnEncoded) {
                 try {
@@ -312,17 +343,19 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             else {
                 // should succeed 
                 conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
-                assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
-                assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
-            
+                assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 6, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "VIEW_COL4", "VIEW_COL2");
+                // even though we added columns to the base table, the view metadata remains the same as the base table metadata changes are no longer propagated to the chid view
+                assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 10, 4, "ID", "COL1", "COL2", "COL3", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+                
                 // query table
                 ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
                 assertTrue(rs.next());
                 assertEquals("view1", rs.getString("ID"));
                 assertEquals(12, rs.getInt("COL1"));
                 assertEquals(13, rs.getInt("COL2"));
-                assertEquals("view5", rs.getString("VIEW_COL2"));
-                assertEquals(17, rs.getInt("VIEW_COL4"));
+                assertEquals("view4", rs.getString("COL3"));
+                assertEquals("view6", rs.getString("VIEW_COL2"));
+                assertEquals(18, rs.getInt("VIEW_COL4"));
                 assertFalse(rs.next());
     
                 // query view
@@ -331,13 +364,20 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                 assertEquals("view1", rs.getString("ID"));
                 assertEquals(12, rs.getInt("COL1"));
                 assertEquals(13, rs.getInt("COL2"));
-                assertEquals(14, rs.getInt("VIEW_COL1"));
-                assertEquals("view5", rs.getString("VIEW_COL2"));
-                assertEquals("view6", rs.getString("VIEW_COL3"));
-                assertEquals(17, rs.getInt("VIEW_COL4"));
-                assertEquals(18, rs.getInt("VIEW_COL5"));
-                assertEquals("view9", rs.getString("VIEW_COL6"));
+                assertEquals("view4", rs.getString("COL3"));
+                assertEquals(15, rs.getInt("VIEW_COL1"));
+                assertEquals("view6", rs.getString("VIEW_COL2"));
+                assertEquals("view7", rs.getString("VIEW_COL3"));
+                assertEquals(18, rs.getInt("VIEW_COL4"));
+                assertEquals(19, rs.getInt("VIEW_COL5"));
+                assertEquals("view10", rs.getString("VIEW_COL6"));
                 assertFalse(rs.next());
+                
+                // the base column count and ordinal positions of columns is updated in the ptable (at read time) 
+                PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
+                PTable view = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable));
+                assertEquals(isMultiTenant ? 5: 4, view.getBaseColumnCount());
+                assertColumnsMatch(view.getColumns(), "ID", "COL1", "COL2", "COL3", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
             }
         } 
     }
@@ -348,8 +388,8 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {      
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
-            String tableName = generateUniqueName();
-            String viewOfTable = tableName + "_VIEW";
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
 
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(10) NOT NULL,"
@@ -359,9 +399,13 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                             + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            PTable table = PhoenixRuntime.getTableNoCache(conn, tableName.toUpperCase());
+            assertColumnsMatch(table.getColumns(), "ID", "COL1", "COL2");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            PTable view = PhoenixRuntime.getTableNoCache(viewConn, viewOfTable.toUpperCase());
+            assertColumnsMatch(view.getColumns(), "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // upsert single row into view
             String dml = "UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?)";
@@ -431,7 +475,8 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             // add the pk column of the view to the base table
             conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            // even though we added columns to the base table, the sequence number and base column count is not updated in the view metadata (in SYSTEM.CATALOG)
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // query table
             ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
@@ -452,6 +497,11 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertEquals(14, rs.getInt("VIEW_COL1"));
             assertEquals("view5", rs.getString("VIEW_COL2"));
             assertFalse(rs.next());
+            
+            // the base column count is updated in the ptable
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
+            view = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable));
+            assertEquals(isMultiTenant ? 4 : 3, view.getBaseColumnCount());
         } 
     }
     
@@ -459,9 +509,10 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public void testAddExistingViewPkColumnToBaseTableWithMultipleViews() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
-            String tableName = generateUniqueName();
-            String viewOfTable1 = tableName + "_VIEW1";
-            String viewOfTable2 = tableName + "_VIEW2";
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String viewOfTable2 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + "("
                             + " %s ID char(10) NOT NULL,"
                             + " COL1 integer NOT NULL,"
@@ -472,10 +523,10 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(viewConn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL3 VARCHAR(256), VIEW_COL4 DECIMAL(10,2) CONSTRAINT pk PRIMARY KEY (VIEW_COL3, VIEW_COL4)) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL3", "VIEW_COL4");
+            assertTableDefinition(viewConn, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL3", "VIEW_COL4");
             
             try {
                 // should fail because there are two view with different pk columns
@@ -523,9 +574,10 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
             viewConn2.setAutoCommit(false);
-            String tableName = generateUniqueName();
-            String viewOfTable1 = tableName + "_VIEW1";
-            String viewOfTable2 = tableName + "_VIEW2";
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String viewOfTable2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + "("
                     + " %s ID char(10) NOT NULL,"
                     + " COL1 integer NOT NULL,"
@@ -536,11 +588,11 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(viewConn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             viewConn2.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
-            assertTableDefinition(conn, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            
+            assertTableDefinition(viewConn2, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3,  "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+
             // upsert single row into both view
             String dml = "UPSERT INTO " + viewOfTable1 + " VALUES(?,?,?,?,?)";
             PreparedStatement stmt = viewConn.prepareStatement(dml);
@@ -590,8 +642,9 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
             assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            assertTableDefinition(conn, viewOfTable1, PTableType.VIEW, tableName, 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            assertTableDefinition(conn, viewOfTable2, PTableType.VIEW, tableName, 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            // even though we added columns to the base table, the sequence number and base column count is not updated in the view metadata (in SYSTEM.CATALOG)
+            assertTableDefinition(viewConn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(viewConn, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // query table
             ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
@@ -620,22 +673,37 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertEquals(14, rs.getInt("VIEW_COL1"));
             assertEquals("view5", rs.getString("VIEW_COL2"));
             assertFalse(rs.next());
+            
+            // the base column count is updated in the ptable
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
+            PTable view1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
+            PTable view2 = viewConn2.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
+            assertEquals(isMultiTenant ? 4 : 3, view1.getBaseColumnCount());
+            assertEquals(isMultiTenant ? 4 : 3, view2.getBaseColumnCount());
         }
     }
     
-    public void assertTableDefinition(Connection conn, String tableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnNames) throws Exception {
+    public void assertTableDefinition(Connection conn, String fullTableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnNames) throws Exception {
         int delta = isMultiTenant ? 1 : 0;
         String[] cols;
-        if (isMultiTenant) {
+        if (isMultiTenant && tableType!=PTableType.VIEW) {
             cols = (String[])ArrayUtils.addAll(new String[]{"TENANT_ID"}, columnNames);
         }
         else {
             cols = columnNames;
         }
-        AlterMultiTenantTableWithViewsIT.assertTableDefinition(conn, tableName, tableType, parentTableName, sequenceNumber, columnCount + delta,
+        AlterMultiTenantTableWithViewsIT.assertTableDefinition(conn, fullTableName, tableType, parentTableName, sequenceNumber, columnCount + delta,
             baseColumnCount==QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT ? baseColumnCount : baseColumnCount +delta, cols);
     }
     
+    public void assertColumnsMatch(List<PColumn> actual, String... expected) {
+        List<String> expectedCols = Lists.newArrayList(expected);
+        if (isMultiTenant) {
+            expectedCols.add(0, "TENANT_ID");
+        }
+        assertEquals(expectedCols, Lists.transform(actual, function));
+    }
+    
     public static String getSystemCatalogEntriesForTable(Connection conn, String tableName, String message) throws Exception {
         StringBuilder sb = new StringBuilder(message);
         sb.append("\n\n\n");
@@ -654,73 +722,78 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     }
     
     
-    
     @Test
     public void testAlteringViewThatHasChildViews() throws Exception {
-        String baseTable = generateUniqueName();
-        String childView = baseTable + "cildView";
-        String grandChildView = baseTable + "grandChildView";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String childView = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String grandChildView = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
         try (Connection conn = DriverManager.getConnection(getUrl());
-                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTable + "  ("
-                    + " %s PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
-                    + " CONSTRAINT NAME_PK PRIMARY KEY (%s PK2)"
-                    + " ) %s";
+                Connection viewConn =
+                        isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn) {
+            String ddlFormat =
+                    "CREATE TABLE IF NOT EXISTS " + baseTable + "  ("
+                            + " %s PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                            + " CONSTRAINT NAME_PK PRIMARY KEY (%s PK2)" + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
 
             String childViewDDL = "CREATE VIEW " + childView + " AS SELECT * FROM " + baseTable;
             viewConn.createStatement().execute(childViewDDL);
+            
+            PTable view = PhoenixRuntime.getTableNoCache(viewConn, childView.toUpperCase());
+            assertColumnsMatch(view.getColumns(), "PK2", "V1", "V2");
 
+            String grandChildViewDDL =
+                    "CREATE VIEW " + grandChildView + " AS SELECT * FROM " + childView;
+            viewConn.createStatement().execute(grandChildViewDDL);
+            
             String addColumnToChildViewDDL =
                     "ALTER VIEW " + childView + " ADD CHILD_VIEW_COL VARCHAR";
             viewConn.createStatement().execute(addColumnToChildViewDDL);
 
-            String grandChildViewDDL =
-                    "CREATE VIEW " + grandChildView + " AS SELECT * FROM " + childView;
-            viewConn.createStatement().execute(grandChildViewDDL);
+            view = PhoenixRuntime.getTableNoCache(viewConn, childView.toUpperCase());
+            assertColumnsMatch(view.getColumns(), "PK2", "V1", "V2", "CHILD_VIEW_COL");
+
+            PTable gcView = PhoenixRuntime.getTableNoCache(viewConn, grandChildView.toUpperCase());
+            assertColumnsMatch(gcView.getColumns(), "PK2", "V1", "V2", "CHILD_VIEW_COL");
 
             // dropping base table column from child view should succeed
             String dropColumnFromChildView = "ALTER VIEW " + childView + " DROP COLUMN V2";
             viewConn.createStatement().execute(dropColumnFromChildView);
+            view = PhoenixRuntime.getTableNoCache(viewConn, childView.toUpperCase());
+            assertColumnsMatch(view.getColumns(), "PK2", "V1", "CHILD_VIEW_COL");
 
             // dropping view specific column from child view should succeed
             dropColumnFromChildView = "ALTER VIEW " + childView + " DROP COLUMN CHILD_VIEW_COL";
             viewConn.createStatement().execute(dropColumnFromChildView);
-            
+            view = PhoenixRuntime.getTableNoCache(viewConn, childView.toUpperCase());
+            assertColumnsMatch(view.getColumns(), "PK2", "V1");
+
             // Adding column to view that has child views is allowed
             String addColumnToChildView = "ALTER VIEW " + childView + " ADD V5 VARCHAR";
             viewConn.createStatement().execute(addColumnToChildView);
-            // V5 column should be visible now for childView
-            viewConn.createStatement().execute("SELECT V5 FROM " + childView);    
-            
-            // However, column V5 shouldn't have propagated to grandChildView. Not till PHOENIX-2054 is fixed.
-            try {
-                viewConn.createStatement().execute("SELECT V5 FROM " + grandChildView);
-            } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-            }
+            // V5 column should be visible now for both childView and grandChildView
+            viewConn.createStatement().execute("SELECT V5 FROM " + childView);
+            viewConn.createStatement().execute("SELECT V5 FROM " + grandChildView);
 
-            // dropping column from the grand child view, however, should work.
-            String dropColumnFromGrandChildView =
-                    "ALTER VIEW " + grandChildView + " DROP COLUMN CHILD_VIEW_COL";
-            viewConn.createStatement().execute(dropColumnFromGrandChildView);
+            view = PhoenixRuntime.getTableNoCache(viewConn, childView.toUpperCase());
+            assertColumnsMatch(view.getColumns(), "PK2", "V1", "V5");
 
-            // similarly, dropping column inherited from the base table should work.
-            dropColumnFromGrandChildView = "ALTER VIEW " + grandChildView + " DROP COLUMN V2";
-            viewConn.createStatement().execute(dropColumnFromGrandChildView);
+            // grand child view should have the same columns
+            gcView = PhoenixRuntime.getTableNoCache(viewConn, grandChildView.toUpperCase());
+            assertColumnsMatch(gcView.getColumns(), "PK2", "V1", "V5");
         }
     }
     
     @Test
     public void testDivergedViewsStayDiverged() throws Exception {
-        String baseTable = generateUniqueName();
-        String view1 = baseTable + "_VIEW1";
-        String view2 = baseTable + "_VIEW2";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String view2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ;
                 Connection viewConn2 = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL2) : conn) {
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTable + " ("
-                    + " %s PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                    + " %s PK1 VARCHAR NOT NULL, V0 VARCHAR, V1 VARCHAR, V2 VARCHAR "
                     + " CONSTRAINT NAME_PK PRIMARY KEY (%s PK1)"
                     + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
@@ -734,8 +807,11 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             // Drop the column inherited from base table to make it diverged
             String dropColumn = "ALTER VIEW " + view1 + " DROP COLUMN V2";
             viewConn.createStatement().execute(dropColumn);
+            PTable table = PhoenixRuntime.getTableNoCache(viewConn, view1);
+            assertEquals(QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT, table.getBaseColumnCount());
             
-            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD V3 VARCHAR";
+            // Add a new regular column and pk column  to the base table
+            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD V3 VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
             
             // Column V3 shouldn't have propagated to the diverged view.
@@ -749,6 +825,43 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             // However, column V3 should have propagated to the non-diverged view.
             sql = "SELECT V3 FROM " + view2;
             viewConn2.createStatement().execute(sql);
+            
+            // PK2 should be in both views
+            sql = "SELECT PK2 FROM " + view1;
+            viewConn.createStatement().execute(sql);
+            sql = "SELECT PK2 FROM " + view2;
+            viewConn2.createStatement().execute(sql);
+            
+            // Drop a column from the base table
+            alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V1";
+            conn.createStatement().execute(alterBaseTable);
+            
+            // V1 should be dropped from both diverged and non-diverged views
+            sql = "SELECT V1 FROM " + view1;
+            try {
+                viewConn.createStatement().execute(sql);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
+            }
+            sql = "SELECT V1 FROM " + view2;
+            try {
+                viewConn2.createStatement().execute(sql);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
+            }
+            
+            // V0 should be still exist in both diverged and non-diverged views
+            sql = "SELECT V0 FROM " + view1;
+            viewConn.createStatement().execute(sql);
+            sql = "SELECT V0 FROM " + view2;
+            viewConn2.createStatement().execute(sql);
+
+			// add the column that was dropped back to the view
+			String addColumn = "ALTER VIEW " + view1 + " ADD V2 VARCHAR";
+			viewConn.createStatement().execute(addColumn);
+			// V2 should not exist in the view
+			sql = "SELECT V0 FROM " + view1;
+			viewConn.createStatement().execute(sql);
         } 
     }
     
@@ -756,8 +869,9 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public void testMakeBaseTableTransactional() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {  
-            String baseTableName = "NONTXNTBL_" + generateUniqueName() + (isMultiTenant ? "0":"1");
-            String viewOfTable = baseTableName + "_VIEW";
+            String baseTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTableName + " ("
                             + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL,"
@@ -768,11 +882,11 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertTableDefinition(conn, baseTableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM "+baseTableName);
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, baseTableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(viewConn, viewOfTable, PTableType.VIEW, baseTableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
-            PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
-            HTableInterface htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
+            Table htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
             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());
@@ -787,50 +901,50 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertTrue(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).isTransactional());
         } 
     }
-    
+
     @Test
     public void testAlterTablePropertyOnView() throws Exception {
-    	try (Connection conn = DriverManager.getConnection(getUrl());
+        try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {  
-            String baseTableName = "NONTXNTBL_" + generateUniqueName() + (isMultiTenant ? "0":"1");
-            String viewOfTable = baseTableName + "_VIEW";
-            
-	        String ddl = "CREATE TABLE " + baseTableName + " (\n"
-	                +"%s ID VARCHAR(15) NOT NULL,\n"
-	                + " COL1 integer NOT NULL,"
-	                +"CREATED_DATE DATE,\n"
-	                +"CONSTRAINT PK PRIMARY KEY (%s ID, COL1)) %s";
-	        conn.createStatement().execute(generateDDL(ddl));
-	        ddl = "CREATE VIEW " + viewOfTable + " AS SELECT * FROM " + baseTableName;
-	        viewConn.createStatement().execute(ddl);
-	        
-	        try {
-	        	viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET IMMUTABLE_ROWS = true");
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW.getErrorCode(), e.getErrorCode());
-	        }
-	        
-        	viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET UPDATE_CACHE_FREQUENCY = 100");
-        	viewConn.createStatement().execute("SELECT * FROM "+ viewOfTable);
-        	PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
-        	assertEquals(100, viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).getUpdateCacheFrequency());
-	        
-	        try {
-	        	viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET APPEND_ONLY_SCHEMA = true");
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW.getErrorCode(), e.getErrorCode());
-	        }
-    	}
+            String baseTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            
+            String ddl = "CREATE TABLE " + baseTableName + " (\n"
+                    +"%s ID VARCHAR(15) NOT NULL,\n"
+                    + " COL1 integer NOT NULL,"
+                    +"CREATED_DATE DATE,\n"
+                    +"CONSTRAINT PK PRIMARY KEY (%s ID, COL1)) %s";
+            conn.createStatement().execute(generateDDL(ddl));
+            ddl = "CREATE VIEW " + viewOfTable + " AS SELECT * FROM " + baseTableName;
+            viewConn.createStatement().execute(ddl);
+            
+            try {
+                viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET IMMUTABLE_ROWS = true");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW.getErrorCode(), e.getErrorCode());
+            }
+            
+            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET UPDATE_CACHE_FREQUENCY = 100");
+            viewConn.createStatement().execute("SELECT * FROM "+ viewOfTable);
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
+            assertEquals(100, viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).getUpdateCacheFrequency());
+            
+            try {
+                viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET APPEND_ONLY_SCHEMA = true");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW.getErrorCode(), e.getErrorCode());
+            }
+        }
     }
     
     @Test
     public void testAlterAppendOnlySchema() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {  
-            String baseTableName = "NONTXNTBL_" + generateUniqueName() + (isMultiTenant ? "0":"1");
-            String viewOfTable = baseTableName + "_VIEW";
+            String baseTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
             
             String ddl = "CREATE TABLE " + baseTableName + " (\n"
                     +"%s ID VARCHAR(15) NOT NULL,\n"
@@ -843,7 +957,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             PTable table = phoenixConn.getTable(new PTableKey(null, baseTableName));
-            PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
             assertFalse(table.isAppendOnlySchema());
             PTable viewTable = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable));
             assertFalse(viewTable.isAppendOnlySchema());
@@ -866,45 +980,112 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             assertTrue(viewTable.isAppendOnlySchema());
         }
     }
-
+    
     @Test
-    public void testAlterTableWithIndexesExtendPk() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.setAutoCommit(false);
-        String tableName = generateUniqueName();
-        String indexName1 = "I_" + generateUniqueName();
-        String indexName2 = "I_" + generateUniqueName();
+    public void testDroppingIndexedColDropsViewIndex() throws Exception {
+        try (Connection conn =DriverManager.getConnection(getUrl());
+                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn  ) {
+            String tableWithView = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String viewSchemaName = SchemaUtil.getSchemaNameFromFullName(viewOfTable);
+            String viewIndex1 = generateUniqueName();
+            String viewIndex2 = generateUniqueName();
+            String fullNameViewIndex1 = SchemaUtil.getTableName(viewSchemaName, viewIndex1);
+            String fullNameViewIndex2 = SchemaUtil.getTableName(viewSchemaName, viewIndex2);
+            
+            conn.setAutoCommit(false);
+            viewConn.setAutoCommit(false);
+            String ddlFormat =
+                    "CREATE TABLE " + tableWithView
+                            + " (%s k VARCHAR NOT NULL, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR, v4 VARCHAR CONSTRAINT PK PRIMARY KEY(%s k))%s";
+            conn.createStatement().execute(generateDDL(ddlFormat));
+            viewConn.createStatement()
+                    .execute(
+                        "CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableWithView );
+            // create an index with the column that will be dropped
+            viewConn.createStatement().execute("CREATE INDEX " + viewIndex1 + " ON " + viewOfTable + "(v2) INCLUDE (v4)");
+            // create an index without the column that will be dropped
+            viewConn.createStatement().execute("CREATE INDEX " + viewIndex2 + " ON " + viewOfTable + "(v1) INCLUDE (v4)");
+            // verify index was created
+            try {
+                viewConn.createStatement().execute("SELECT * FROM " + fullNameViewIndex1 );
+            } catch (TableNotFoundException e) {
+                fail("Index on view was not created");
+            }
+            
+            // upsert a single row
+            PreparedStatement stmt = viewConn.prepareStatement("UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "b");
+            stmt.setString(3, "c");
+            stmt.setString(4, "d");
+            stmt.setString(5, "e");
+            stmt.setInt(6, 1);
+            stmt.setString(7, "g");
+            stmt.execute();
+            viewConn.commit();
 
-        try {
-            String ddl = "CREATE TABLE " + tableName +
-            " (ORG_ID CHAR(15) NOT NULL," +
-            " PARTITION_KEY CHAR(3) NOT NULL, " +
-            " ACTIVITY_DATE DATE NOT NULL, " +
-            " FK1_ID CHAR(15) NOT NULL, " +
-            " FK2_ID CHAR(15) NOT NULL, " +
-            " TYPE VARCHAR NOT NULL, " +
-            " IS_OPEN BOOLEAN " +
-            " CONSTRAINT PKVIEW PRIMARY KEY " +
-            "(" +
-            "ORG_ID, PARTITION_KEY, ACTIVITY_DATE, FK1_ID, FK2_ID, TYPE" +
-            "))";
-            createTestTable(getUrl(), ddl);
-            
-            String idx1ddl = "CREATE INDEX " + indexName1 + " ON " + tableName + " (FK1_ID, ACTIVITY_DATE DESC) INCLUDE (IS_OPEN)";
-            PreparedStatement stmt1 = conn.prepareStatement(idx1ddl);
-            stmt1.execute();
-            
-            String idx2ddl = "CREATE INDEX " + indexName2 + " ON " + tableName + " (FK2_ID, ACTIVITY_DATE DESC) INCLUDE (IS_OPEN)";
-            PreparedStatement stmt2 = conn.prepareStatement(idx2ddl);
-            stmt2.execute();
-        
-            ddl = "ALTER TABLE " + tableName + " ADD SOURCE VARCHAR(25) NULL PRIMARY KEY";
-            PreparedStatement stmt3 = conn.prepareStatement(ddl);
-            stmt3.execute();
-        } finally {
-            conn.close();
+            // verify the index was created
+            PhoenixConnection pconn = viewConn.unwrap(PhoenixConnection.class);
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null; 
+            PTable view = pconn.getTable(new PTableKey(tenantId,  viewOfTable ));
+            PTable viewIndex = pconn.getTable(new PTableKey(tenantId,  fullNameViewIndex1 ));
+            byte[] viewIndexPhysicalTable = viewIndex.getPhysicalName().getBytes();
+            assertNotNull("Can't find view index", viewIndex);
+            assertEquals("Unexpected number of indexes ", 2, view.getIndexes().size());
+            assertEquals("Unexpected index ",  fullNameViewIndex1 , view.getIndexes().get(0).getName()
+                    .getString());
+            assertEquals("Unexpected index ",  fullNameViewIndex2 , view.getIndexes().get(1).getName()
+                .getString());
+            
+            // drop two columns
+            conn.createStatement().execute("ALTER TABLE " + tableWithView + " DROP COLUMN v2, v3 ");
+            
+            // verify columns were dropped
+            try {
+                conn.createStatement().execute("SELECT v2 FROM " + tableWithView );
+                fail("Column should have been dropped");
+            } catch (ColumnNotFoundException e) {
+            }
+            try {
+                conn.createStatement().execute("SELECT v3 FROM " + tableWithView );
+                fail("Column should have been dropped");
+            } catch (ColumnNotFoundException e) {
+            }
+            
+            // verify index metadata was dropped
+            try {
+                viewConn.createStatement().execute("SELECT * FROM " + fullNameViewIndex1 );
+                fail("Index metadata should have been dropped");
+            } catch (TableNotFoundException e) {
+            }
+            
+            pconn = viewConn.unwrap(PhoenixConnection.class);
+            view = pconn.getTable(new PTableKey(tenantId,  viewOfTable ));
+            try {
+                viewIndex = pconn.getTable(new PTableKey(tenantId,  fullNameViewIndex1 ));
+                fail("View index should have been dropped");
+            } catch (TableNotFoundException e) {
+            }
+            assertEquals("Unexpected number of indexes ", 1, view.getIndexes().size());
+            assertEquals("Unexpected index ",  fullNameViewIndex2 , view.getIndexes().get(0).getName().getString());
+            
+            // verify that the physical index view table is *not* dropped
+            conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(viewIndexPhysicalTable);
+            
+            // scan the physical table and verify there is a single row for the second local index
+            Scan scan = new Scan();
+            HTable table = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(viewIndexPhysicalTable);
+            ResultScanner results = table.getScanner(scan);
+            Result result = results.next();
+            assertNotNull(result);
+            PTable viewIndexPTable = pconn.getTable(new PTableKey(pconn.getTenantId(), fullNameViewIndex2));
+            PColumn column = viewIndexPTable.getColumnForColumnName(IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V4"));
+            byte[] cq = column.getColumnQualifierBytes();
+            // there should be a single row belonging to VIEWINDEX2 
+            assertNotNull(fullNameViewIndex2 + " row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, cq));
+            assertNull(results.next());
         }
     }
-
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 d601beb..b39c4f0 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
@@ -109,7 +109,9 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT {
             }
             
             // verify getTable rpcs
-            verify(connectionQueryServices, sameClient ? never() : times(1)).getTable((PName)isNull(), eq(new byte[0]), eq(Bytes.toBytes(viewName)), anyLong(), anyLong());
+            verify(connectionQueryServices, sameClient ? never() : times(1)).getTable(
+                (PName) isNull(), eq(new byte[0]), eq(Bytes.toBytes(viewName)), anyLong(),
+                anyLong(), eq(false), eq(false), (PTable) isNull());
             
             // verify no create table rpcs
             verify(connectionQueryServices, never()).createTable(anyListOf(Mutation.class),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
index c61d970..9bd689e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
@@ -32,13 +32,12 @@ import java.util.Set;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.collect.Lists;
 
-public class BaseTenantSpecificViewIndexIT extends ParallelStatsDisabledIT {
+public class BaseTenantSpecificViewIndexIT extends SplitSystemCatalogIT {
     
-    public static final String TENANT1_ID = "tenant1";
-    public static final String TENANT2_ID = "tenant2";
     public static final String NON_STRING_TENANT_ID = "1234";
     
     protected Set<Pair<String, String>> tenantViewsToDelete = newHashSet();
@@ -48,12 +47,13 @@ public class BaseTenantSpecificViewIndexIT extends ParallelStatsDisabledIT {
     }
     
     protected void testUpdatableView(Integer saltBuckets, boolean localIndex) throws Exception {
-        String tableName = generateUniqueName();
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         createBaseTable(tableName, saltBuckets, true);
-        Connection conn = createTenantConnection(TENANT1_ID);
+        Connection conn = createTenantConnection(TENANT1);
         try {
-            String viewName = createAndPopulateTenantView(conn, TENANT1_ID, tableName, "");
-            createAndVerifyIndex(conn, viewName, tableName, saltBuckets, TENANT1_ID, "", localIndex);
+            createAndPopulateTenantView(conn, TENANT1, tableName, "", viewName);
+            createAndVerifyIndex(conn, viewName, tableName, saltBuckets, TENANT1, "", localIndex);
             verifyViewData(conn, viewName, "");
         } finally {
             try { conn.close();} catch (Exception ignored) {}
@@ -61,11 +61,12 @@ public class BaseTenantSpecificViewIndexIT extends ParallelStatsDisabledIT {
     }
 
     protected void testUpdatableViewNonString(Integer saltBuckets, boolean localIndex) throws Exception {
-        String tableName = generateUniqueName();
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         createBaseTable(tableName, saltBuckets, false);
         Connection conn = createTenantConnection(NON_STRING_TENANT_ID);
         try {
-            String viewName = createAndPopulateTenantView(conn, NON_STRING_TENANT_ID, tableName, "");
+            createAndPopulateTenantView(conn, NON_STRING_TENANT_ID, tableName, "", viewName);
             createAndVerifyIndexNonStringTenantId(conn, viewName, tableName, NON_STRING_TENANT_ID, "");
             verifyViewData(conn, viewName, "");
         } finally {
@@ -78,20 +79,22 @@ public class BaseTenantSpecificViewIndexIT extends ParallelStatsDisabledIT {
     }
 
     protected void testUpdatableViewsWithSameNameDifferentTenants(Integer saltBuckets, boolean localIndex) throws Exception {
-        String tableName = generateUniqueName();
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String viewName2 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
         createBaseTable(tableName, saltBuckets, true);
-        Connection conn1 = createTenantConnection(TENANT1_ID);
-        Connection conn2 = createTenantConnection(TENANT2_ID);
+        Connection conn1 = createTenantConnection(TENANT1);
+        Connection conn2 = createTenantConnection(TENANT2);
         try {
             String prefixForTenant1Data = "TI";
             String prefixForTenant2Data = "TII";
             
             // tenant views with same name for two different tables
-            String viewName1 = createAndPopulateTenantView(conn1, TENANT1_ID, tableName, prefixForTenant1Data);
-            String viewName2 = createAndPopulateTenantView(conn2, TENANT2_ID, tableName, prefixForTenant2Data);
+            createAndPopulateTenantView(conn1, TENANT1, tableName, prefixForTenant1Data, viewName1);
+            createAndPopulateTenantView(conn2, TENANT2, tableName, prefixForTenant2Data, viewName2);
             
-            createAndVerifyIndex(conn1, viewName1, tableName, saltBuckets, TENANT1_ID, prefixForTenant1Data, localIndex);
-            createAndVerifyIndex(conn2, viewName2, tableName, saltBuckets, TENANT2_ID, prefixForTenant2Data, localIndex);
+            createAndVerifyIndex(conn1, viewName1, tableName, saltBuckets, TENANT1, prefixForTenant1Data, localIndex);
+            createAndVerifyIndex(conn2, viewName2, tableName, saltBuckets, TENANT2, prefixForTenant2Data, localIndex);
             
             verifyViewData(conn1, viewName1, prefixForTenant1Data);
             verifyViewData(conn2, viewName2, prefixForTenant2Data);
@@ -114,8 +117,7 @@ public class BaseTenantSpecificViewIndexIT extends ParallelStatsDisabledIT {
         conn.close();
     }
     
-    private String createAndPopulateTenantView(Connection conn, String tenantId, String baseTable, String valuePrefix) throws SQLException {
-        String viewName = generateUniqueName();
+    private String createAndPopulateTenantView(Connection conn, String tenantId, String baseTable, String valuePrefix, String viewName) throws SQLException {
         String ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR) AS SELECT * FROM " + baseTable + " WHERE k1 = 1";
         conn.createStatement().execute(ddl);
         tenantViewsToDelete.add(new Pair<String, String>(tenantId, viewName ));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index abaa2f6..0dc4410 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -692,7 +692,7 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             assertTrue(info.getEstimateInfoTs() > 0);
 
             conn.createStatement()
-                    .execute("ALTER TABLE " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+                    .execute("ALTER VIEW " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
             sql = "SELECT COUNT(*) FROM " + viewName;
             // query the view
             rs = conn.createStatement().executeQuery(sql);
@@ -1202,33 +1202,44 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
         assertEquals("B", rs.getString(1));
     }
 
-	@Test
-	public void testUseStatsForParallelizationProperyOnViewIndex() throws SQLException {
-		String tableName = generateUniqueName();
-		String viewName = generateUniqueName();
-		String tenantViewName = generateUniqueName();
-		String viewIndexName = generateUniqueName();
-		boolean useStats = !DEFAULT_USE_STATS_FOR_PARALLELIZATION;
-		try (Connection conn = DriverManager.getConnection(getUrl())) {
-			conn.createStatement()
-					.execute("create table " + tableName
-							+ "(tenantId CHAR(15) NOT NULL, pk1 integer NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
-							+ "(tenantId, pk1)) MULTI_TENANT=true");
-			try (Connection tenantConn = getTenantConnection("tenant1")) {
-				conn.createStatement().execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName);
-				conn.createStatement().execute("CREATE INDEX " + viewIndexName + " on " + viewName + " (v) ");
-				tenantConn.createStatement().execute("CREATE VIEW " + tenantViewName + " AS SELECT * FROM " + viewName);
-				conn.createStatement()
-						.execute("ALTER TABLE " + tableName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
-				// fetch the latest view ptable 
-				PhoenixRuntime.getTableNoCache(tenantConn, viewName);
-				PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
-				PTable viewIndex = phxConn.getTable(new PTableKey(phxConn.getTenantId(), viewIndexName));
-				assertEquals("USE_STATS_FOR_PARALLELIZATION property set incorrectly", useStats,
-						PhoenixConfigurationUtil
-								.getStatsForParallelizationProp(tenantConn.unwrap(PhoenixConnection.class), viewIndex));
-			}
-		}
-	}
+    @Test
+    public void testUseStatsForParallelizationProperyOnViewIndex() throws SQLException {
+        String tableName = generateUniqueName();
+        String viewName = generateUniqueName();
+        String tenantViewName = generateUniqueName();
+        String viewIndexName = generateUniqueName();
+        boolean useStats = !DEFAULT_USE_STATS_FOR_PARALLELIZATION;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement()
+                    .execute("create table " + tableName
+                            + "(tenantId CHAR(15) NOT NULL, pk1 integer NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
+                            + "(tenantId, pk1)) MULTI_TENANT=true");
+            try (Connection tenantConn = getTenantConnection("tenant1")) {
+                conn.createStatement().execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName);
+                conn.createStatement().execute("CREATE INDEX " + viewIndexName + " on " + viewName + " (v) ");
+                tenantConn.createStatement().execute("CREATE VIEW " + tenantViewName + " AS SELECT * FROM " + viewName);
+                conn.createStatement()
+                        .execute("ALTER TABLE " + tableName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
+                // changing a property on a base table does not change the property on a view
+                validatePropertyOnViewIndex(viewName, viewIndexName, !useStats, conn, tenantConn);
+
+                // need to explicitly change the property on the view
+                conn.createStatement()
+                        .execute("ALTER VIEW " + viewName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
+                validatePropertyOnViewIndex(viewName, viewIndexName, useStats, conn, tenantConn);
+            }
+        }
+    }
+
+    private void validatePropertyOnViewIndex(String viewName, String viewIndexName, boolean useStats, Connection conn,
+            Connection tenantConn) throws SQLException, TableNotFoundException {
+        // fetch the latest view ptable
+        PhoenixRuntime.getTableNoCache(tenantConn, viewName);
+        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+        PTable viewIndex = phxConn.getTable(new PTableKey(phxConn.getTenantId(), viewIndexName));
+        assertEquals("USE_STATS_FOR_PARALLELIZATION property set incorrectly", useStats,
+                PhoenixConfigurationUtil
+                        .getStatsForParallelizationProp(tenantConn.unwrap(PhoenixConnection.class), viewIndex));
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 627e453..d253f6e 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
@@ -63,10 +63,10 @@ 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.LOG"));
+            "SYSTEM.MUTEX","SYSTEM.LOG", "SYSTEM.CHILD_LINK"));
     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"));
+                    "SYSTEM:MUTEX","SYSTEM:LOG", "SYSTEM:CHILD_LINK"));
     private static final String SCHEMA_NAME = "MIGRATETEST";
     private static final String TABLE_NAME =
             SCHEMA_NAME + "." + MigrateSystemTablesToSystemNamespaceIT.class.getSimpleName().toUpperCase();
@@ -86,12 +86,10 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
     final UserGroupInformation user4 =
             UserGroupInformation.createUserForTesting("user4", new String[0]);
 
-
-    @Before
-    public final void doSetup() throws Exception {
+    public final void doSetup(boolean systemMappingEnabled) throws Exception {
         testUtil = new HBaseTestingUtility();
         Configuration conf = testUtil.getConfiguration();
-        enableNamespacesOnServer(conf);
+        enableNamespacesOnServer(conf, systemMappingEnabled);
         configureRandomHMasterPort(conf);
         testUtil.startMiniCluster(1);
     }
@@ -110,9 +108,9 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
 
     // Tests that client can create and read tables on a fresh HBase cluster with
     // system namespace mapping enabled from the start
-    @Test
-    public void freshClientsCreateNamespaceMappedSystemTables() throws IOException, InterruptedException {
-
+	@Test
+	public void freshClientsCreateNamespaceMappedSystemTables() throws Exception {
+        doSetup(true);
         user1.doAs(new PrivilegedExceptionAction<Void>() {
             @Override
             public Void run() throws Exception {
@@ -137,9 +135,9 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
     }
 
     // Tests that NEWER clients can read tables on HBase cluster after system tables are migrated
-    @Test
-    public void migrateSystemTablesInExistingCluster() throws IOException, InterruptedException {
-
+	@Test
+	public void migrateSystemTablesInExistingCluster() throws Exception {
+		doSetup(false);
         user1.doAs(new PrivilegedExceptionAction<Void>() {
             @Override
             public Void run() throws Exception {
@@ -167,9 +165,9 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
 
     // Tests that OLDER clients fail after system tables are migrated
     // Clients should be restarted with new properties which are consistent on both client and server
-    @Test
-    public void oldClientsAfterSystemTableMigrationShouldFail() throws IOException, InterruptedException {
-
+	@Test
+	public void oldClientsAfterSystemTableMigrationShouldFail() throws Exception {
+        doSetup(true);
         user1.doAs(new PrivilegedExceptionAction<Void>() {
             @Override
             public Void run() throws Exception {
@@ -202,9 +200,9 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
 
     // Tests that only one client can migrate the system table to system namespace
     // Migrate process acquires lock in SYSMUTEX table
-    @Test
-    public void onlyOneClientCanMigrate() throws IOException, InterruptedException, SQLException {
-
+	@Test
+	public void onlyOneClientCanMigrate() throws Exception {
+        doSetup(false);
         user1.doAs(new PrivilegedExceptionAction<Void>() {
             @Override
             public Void run() throws Exception {
@@ -291,8 +289,10 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
         }
     }
 
-    private void enableNamespacesOnServer(Configuration conf) {
+    private void enableNamespacesOnServer(Configuration conf, boolean systemMappingEnabled) {
         conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());
+		conf.set(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE,
+				systemMappingEnabled ? Boolean.TRUE.toString() : Boolean.FALSE.toString());
     }
 
     // For PHOENIX-4389 (Flapping tests SystemTablePermissionsIT and MigrateSystemTablesToSystemNamespaceIT)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
index c93d2aa..255a8b2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -174,30 +174,24 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
         String schemaName = "S_" + generateUniqueName();
         String tableName = "T_" + generateUniqueName();
         String phoenixFullTableName = SchemaUtil.getTableName(schemaName, tableName);
-        String viewName1 = "VC_" + generateUniqueName();
-        String viewName2 = "VB_" + generateUniqueName();
+        String viewName1 = "VB_" + generateUniqueName();
+        String viewName2 = "VC_" + generateUniqueName();
 
         try (Connection conn = createConnection(null, false)) {
             conn.createStatement().execute("CREATE TABLE " + phoenixFullTableName
                     + "(k VARCHAR not null, v INTEGER not null, f INTEGER, g INTEGER NULL, h INTEGER NULL CONSTRAINT pk PRIMARY KEY(k,v)) MULTI_TENANT=true");
         }
-        try (Connection conn = createConnection("tenant1", false)) {
-            // create view
-            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName1
-                    + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
-            // create child view
-            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName2
-                    + " (col2 VARCHAR) AS SELECT * FROM " + schemaName + "." + viewName1);
-        }
 
-        String tenant2 = "tenant2";
-        try (Connection conn = createConnection(tenant2, false)) {
-            // creating another view in a second tenant but same view name
-            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName1
-                    + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
-            // creating child view with a second tenant
-            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName2
-                    + " (col2 VARCHAR) AS SELECT * FROM " + schemaName + "." + viewName1);
+        String[] tenantIds = new String[] { "tenant1", "tenant2" };
+        for (String tenantId : tenantIds) {
+            try (Connection conn = createConnection(tenantId, false)) {
+                // create view
+                conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName1
+                        + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+                // create child view
+                conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName2
+                        + " (col2 VARCHAR) AS SELECT * FROM " + schemaName + "." + viewName1);
+            }
         }
 
         try (Connection conn = createConnection(null, true)) {
@@ -209,14 +203,15 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
             try (PhoenixConnection phxConn =
                     DriverManager.getConnection(url, props).unwrap(PhoenixConnection.class)) {
                 UpgradeUtil.upgradeTable(phxConn, phoenixFullTableName);
-                UpgradeUtil.mapChildViewsToNamespace(phxConn, phoenixFullTableName, props);
             }
 
             // verify physical table link
             String physicalTableName =
                     SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, true).getString();
-            assertEquals(physicalTableName, getPhysicalTable(conn, tenant2, schemaName, viewName1));
-            assertEquals(physicalTableName, getPhysicalTable(conn, tenant2, schemaName, viewName2));
+            for (String tenantId : tenantIds) {
+                assertEquals(physicalTableName, getPhysicalTable(conn, tenantId, schemaName, viewName1));
+                assertEquals(physicalTableName, getPhysicalTable(conn, tenantId, schemaName, viewName2));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 4f84304..90f9db6 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
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
@@ -162,6 +163,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(SYSTEM_CHILD_LINK_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(SYSTEM_FUNCTION_TABLE, rs.getString("TABLE_NAME"));
             assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
             assertTrue(rs.next());
@@ -342,7 +347,7 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
     @Test
     public void testSchemaMetadataScan() throws SQLException {
         String table1 = generateUniqueName();
-        String schema1 = generateUniqueName();
+        String schema1 = "Z_" + generateUniqueName();
         String fullTable1 = schema1 + "." + table1;
         ensureTableCreated(getUrl(), fullTable1, CUSTOM_ENTITY_DATA_FULL_NAME, null);
         String fullTable2 = generateUniqueName();
@@ -1095,7 +1100,7 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
         // Retrieve the database metadata
         DatabaseMetaData dbmd = conn.getMetaData();
         ResultSet rs = dbmd.getColumns(null, null, null, null);
-        rs.next();
+        assertTrue(rs.next());
 
         // Lookup column by name, this should return null but not throw an exception
         String remarks = rs.getString("REMARKS");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
deleted file mode 100644
index e7518f6..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
+++ /dev/null
@@ -1,45 +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.end2end;
-
-import org.junit.Test;
-
-
-public class SaltedViewIT extends BaseViewIT {
-
-	public SaltedViewIT(boolean transactional) {
-		super(transactional);
-	}
-
-	/**
-     * Salted tests must be in their own test file to ensure that the underlying
-     * table is dropped. Otherwise, the splits may not be performed.
-     * TODO: we should throw in that case
-     * 
-     * @throws Exception
-     */
-    @Test
-    public void testSaltedUpdatableViewWithIndex() throws Exception {
-        testUpdatableViewWithIndex(3, false);
-    }
-
-    @Test
-    public void testSaltedUpdatableViewWithLocalIndex() throws Exception {
-        testUpdatableViewWithIndex(3, true);
-    }
-}


[15/50] [abbrv] phoenix git commit: PHOENIX-3991 ROW_TIMESTAMP on TIMESTAMP column type throws ArrayOutOfBound when upserting without providing a value.

Posted by ja...@apache.org.
PHOENIX-3991 ROW_TIMESTAMP on TIMESTAMP column type throws ArrayOutOfBound when upserting without providing a value.


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

Branch: refs/heads/omid2
Commit: b37b7d7505570d8b198ecd4b6e5a626c73bf7ebe
Parents: 1f7e320
Author: Sergey Soldatov <ss...@apache.org>
Authored: Wed Jul 25 12:48:03 2018 -0700
Committer: Sergey Soldatov <ss...@apache.org>
Committed: Thu Jul 26 13:39:00 2018 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/RowTimestampIT.java  | 26 +++++++++++++++++---
 .../apache/phoenix/execute/MutationState.java   |  6 ++++-
 2 files changed, 27 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b37b7d75/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowTimestampIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowTimestampIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowTimestampIT.java
index 458cc38..0457bf3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowTimestampIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowTimestampIT.java
@@ -73,13 +73,22 @@ public class RowTimestampIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testWithUpsertingRowTimestampColSpecified() throws Exception {
+    public void testUpsertingRowTimestampColSpecifiedWithTimestamp() throws Exception {
+        upsertingRowTimestampColSpecified("TIMESTAMP");
+    }
+
+    @Test
+    public void testUpsertingRowTimestampColSpecifiedWithDate() throws Exception {
+        upsertingRowTimestampColSpecified("DATE");
+    }
+
+    private void upsertingRowTimestampColSpecified(String type) throws Exception {
         String tableName = generateUniqueName();
         String indexName = generateUniqueName();
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement()
                     .execute("CREATE TABLE IF NOT EXISTS " + tableName
-                            + " (PK1 VARCHAR NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 "
+                            + " (PK1 VARCHAR NOT NULL, PK2 " + type + " NOT NULL, KV1 VARCHAR, KV2 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 "
                             + sortOrder + " ROW_TIMESTAMP)) " + tableDDLOptions);
         }
         try (Connection conn = DriverManager.getConnection(getUrl())) {
@@ -192,14 +201,23 @@ public class RowTimestampIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testAutomaticallySettingRowTimestampForImmutableTableAndIndexes() throws Exception {
+    public void testAutomaticallySettingRowTimestampWithTimestamp () throws Exception {
+        automaticallySettingRowTimestampForImmutableTableAndIndexes("TIMESTAMP");
+    }
+
+    @Test
+    public void testAutomaticallySettingRowTimestampWithDate () throws Exception {
+        automaticallySettingRowTimestampForImmutableTableAndIndexes("DATE");
+    }
+
+    private void automaticallySettingRowTimestampForImmutableTableAndIndexes(String type) throws Exception {
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
         String tableName = generateUniqueName();
         String indexName = generateUniqueName();
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement()
                     .execute("CREATE TABLE IF NOT EXISTS " + tableName
-                            + " (PK1 VARCHAR NOT NULL, PK2 DATE NOT NULL, KV1 VARCHAR, KV2 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 "
+                            + " (PK1 VARCHAR NOT NULL, PK2 " + type + " NOT NULL, KV1 VARCHAR, KV2 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2 "
                             + sortOrder + " ROW_TIMESTAMP)) " + tableDDLOptions);
         }
         try (Connection conn = DriverManager.getConnection(getUrl())) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b37b7d75/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 c29d6b5..d2d1eea 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
@@ -25,6 +25,7 @@ import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_
 
 import java.io.IOException;
 import java.sql.SQLException;
+import java.sql.Timestamp;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -85,6 +86,7 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.PhoenixTransactionContext.PhoenixVisibilityLevel;
@@ -474,7 +476,9 @@ public class MutationState implements SQLCloseable {
         RowKeySchema schema = table.getRowKeySchema();
         int rowTimestampColPos = table.getRowTimestampColPos();
         Field rowTimestampField = schema.getField(rowTimestampColPos);
-        byte[] rowTimestampBytes = PLong.INSTANCE.toBytes(rowTimestamp, rowTimestampField.getSortOrder());
+        byte[] rowTimestampBytes = rowTimestampField.getDataType() == PTimestamp.INSTANCE ?
+            PTimestamp.INSTANCE.toBytes(new Timestamp(rowTimestamp), rowTimestampField.getSortOrder()) :
+            PLong.INSTANCE.toBytes(rowTimestamp, rowTimestampField.getSortOrder());
         int oldOffset = ptr.getOffset();
         int oldLength = ptr.getLength();
         // Move the pointer to the start byte of the row timestamp pk


[32/50] [abbrv] phoenix git commit: PHOENIX-4839 IndexHalfStoreFileReaderGenerator throws NullPointerException(Aman Poonia)

Posted by ja...@apache.org.
PHOENIX-4839 IndexHalfStoreFileReaderGenerator throws NullPointerException(Aman Poonia)


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

Branch: refs/heads/omid2
Commit: a575ac04ea4b4497209a194169d2121e210ec307
Parents: 0639a74
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Aug 21 11:51:50 2018 -0700
Committer: Ankit Singhal <an...@gmail.com>
Committed: Tue Aug 21 11:51:50 2018 -0700

----------------------------------------------------------------------
 .../regionserver/IndexHalfStoreFileReader.java  |   6 +
 .../IndexHalfStoreFileReaderGenerator.java      | 138 ++-----------------
 2 files changed, 18 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a575ac04/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
index d1d12fb..8bd0d72 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
@@ -123,4 +123,10 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
     public boolean isTop() {
         return top;
     }
+
+    @Override
+    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread, boolean isCompaction, long readPt) {
+        return new LocalIndexStoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), true,
+                getHFileReader().hasMVCCInfo(), readPt);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a575ac04/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index e41086b..ab65456 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -17,16 +17,11 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_START_ROW_SUFFIX;
-
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -71,7 +66,7 @@ import org.apache.phoenix.util.RepairUtil;
 import com.google.common.collect.Lists;
 
 public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
-    
+
     private static final String LOCAL_INDEX_AUTOMATIC_REPAIR = "local.index.automatic.repair";
     public static final Log LOG = LogFactory.getLog(IndexHalfStoreFileReaderGenerator.class);
 
@@ -153,7 +148,9 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
             try {
                 conn = QueryUtil.getConnectionOnServer(ctx.getEnvironment().getConfiguration()).unwrap(
                             PhoenixConnection.class);
-                PTable dataTable = IndexUtil.getPDataTable(conn, ctx.getEnvironment().getRegion().getTableDesc());
+                PTable dataTable =
+                        IndexUtil.getPDataTable(conn, ctx.getEnvironment().getRegion()
+                                .getTableDesc());
                 List<PTable> indexes = dataTable.getIndexes();
                 Map<ImmutableBytesWritable, IndexMaintainer> indexMaintainers =
                         new HashMap<ImmutableBytesWritable, IndexMaintainer>();
@@ -187,19 +184,12 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
         return reader;
     }
 
-    @SuppressWarnings("deprecation")
     @Override
-    public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
-            Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
-            long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException {
+    public InternalScanner preCompact(
+            ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+            InternalScanner s, ScanType scanType,
+            CompactionRequest request) throws IOException {
         if (!IndexUtil.isLocalIndexStore(store)) { return s; }
-        Scan scan = null;
-        if (s!=null) {
-        	scan = ((StoreScanner)s).scan;
-        } else  {
-        	scan = new Scan();
-        	scan.setMaxVersions(store.getFamily().getMaxVersions());
-        }
         if (!store.hasReferences()) {
             InternalScanner repairScanner = null;
             if (request.isMajor() && (!RepairUtil.isLocalIndexStoreFilesConsistent(c.getEnvironment(), store))) {
@@ -220,23 +210,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
                 return s;
             }
         }
-        List<StoreFileScanner> newScanners = new ArrayList<StoreFileScanner>(scanners.size());
-        boolean scanUsePread = c.getEnvironment().getConfiguration().getBoolean("hbase.storescanner.use.pread", scan.isSmall());
-        for(KeyValueScanner scanner: scanners) {
-            Reader reader = ((StoreFileScanner) scanner).getReader();
-            if (reader instanceof IndexHalfStoreFileReader) {
-                newScanners.add(new LocalIndexStoreFileScanner(reader, reader.getScanner(
-                    scan.getCacheBlocks(), scanUsePread, false), true, reader.getHFileReader()
-                        .hasMVCCInfo(), store.getSmallestReadPoint()));
-            } else {
-                newScanners.add(((StoreFileScanner) scanner));
-            }
-        }
-        if (s!=null) {
-            s.close();
-        }
-        return new StoreScanner(store, store.getScanInfo(), scan, newScanners,
-            scanType, store.getSmallestReadPoint(), earliestPutTs);
+       return s;
     }
 
     private byte[][] getViewConstants(PTable dataTable) {
@@ -269,10 +243,10 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
         }
         return viewConstants;
     }
-    
+
     /**
      * @param env
-     * @param store Local Index store 
+     * @param store Local Index store
      * @param scan
      * @param scanType
      * @param earliestPutTs
@@ -302,100 +276,12 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
             }
             return new DataTableLocalIndexRegionScanner(env.getRegion().getScanner(scan), env.getRegion(),
                     maintainers, store.getFamily().getName(),env.getConfiguration());
-            
+
 
         } catch (ClassNotFoundException | SQLException e) {
             throw new IOException(e);
 
         }
     }
-    
-    @Override
-    public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
-        final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
-        final KeyValueScanner s) throws IOException {
-        if (store.getFamily().getNameAsString()
-                .startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)
-                && store.hasReferences()) {
-            final long readPt = c.getEnvironment().getRegion().getReadpoint(scan.getIsolationLevel
-                    ());
-            if (s!=null) {
-                s.close();
-            }
-            if (!scan.isReversed()) {
-                return new StoreScanner(store, store.getScanInfo(), scan,
-                        targetCols, readPt) {
 
-                    @Override
-                    protected List<KeyValueScanner> getScannersNoCompaction() throws IOException {
-                        if (store.hasReferences()) {
-                            return getLocalIndexScanners(c, store, scan, readPt);
-                        } else {
-                            return super.getScannersNoCompaction();
-                        }
-                    }
-                };
-            } else {
-                return new ReversedStoreScanner(store, store.getScanInfo(), scan,
-                        targetCols, readPt) {
-                    @Override
-                    protected List<KeyValueScanner> getScannersNoCompaction() throws IOException {
-                        if (store.hasReferences()) {
-                            return getLocalIndexScanners(c, store, scan, readPt);
-                        } else {
-                            return super.getScannersNoCompaction();
-                        }
-                    }
-                };
-            }
-        }
-        return s;
-    }
-
-    private List<KeyValueScanner> getLocalIndexScanners(final
-                                                ObserverContext<RegionCoprocessorEnvironment> c,
-                          final Store store, final Scan scan, final long readPt) throws IOException {
-
-        boolean scanUsePread = c.getEnvironment().getConfiguration().getBoolean("hbase.storescanner.use.pread", scan.isSmall());
-        Collection<StoreFile> storeFiles = store.getStorefiles();
-        List<StoreFile> nonReferenceStoreFiles = new ArrayList<>(store.getStorefiles().size());
-        List<StoreFile> referenceStoreFiles = new ArrayList<>(store.getStorefiles().size
-                ());
-        final List<KeyValueScanner> keyValueScanners = new ArrayList<>(store
-                .getStorefiles().size() + 1);
-        byte[] startKey = c.getEnvironment().getRegionInfo().getStartKey();
-        byte[] endKey = c.getEnvironment().getRegionInfo().getEndKey();
-        // If the region start key is not the prefix of the scan start row then we can return empty
-        // scanners. This is possible during merge where one of the child region scan should not return any
-        // results as we go through merged region.
-        int prefixLength =
-                scan.getAttribute(SCAN_START_ROW_SUFFIX) == null ? (startKey.length == 0 ? endKey.length
-                        : startKey.length) : (scan.getStartRow().length - scan.getAttribute(SCAN_START_ROW_SUFFIX).length);
-        if (Bytes.compareTo(scan.getStartRow(), 0, prefixLength, (startKey.length == 0 ? new byte[endKey.length] : startKey), 0,
-            startKey.length == 0 ? endKey.length : startKey.length) != 0) {
-            return keyValueScanners;
-        }
-        for (StoreFile storeFile : storeFiles) {
-            if (storeFile.isReference()) {
-                referenceStoreFiles.add(storeFile);
-            } else {
-                nonReferenceStoreFiles.add(storeFile);
-            }
-        }
-        final List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(nonReferenceStoreFiles, scan.getCacheBlocks(), scanUsePread, readPt);
-        keyValueScanners.addAll(scanners);
-        for (StoreFile sf : referenceStoreFiles) {
-            if (sf.getReader() instanceof IndexHalfStoreFileReader) {
-                keyValueScanners.add(new LocalIndexStoreFileScanner(sf.getReader(), sf.getReader()
-                        .getScanner(scan.getCacheBlocks(), scanUsePread, false), true, sf
-                        .getReader().getHFileReader().hasMVCCInfo(), readPt));
-            } else {
-                keyValueScanners.add(new StoreFileScanner(sf.getReader(), sf.getReader()
-                        .getScanner(scan.getCacheBlocks(), scanUsePread, false), true, sf
-                        .getReader().getHFileReader().hasMVCCInfo(), readPt));
-            }
-        }
-        keyValueScanners.addAll(((HStore) store).memstore.getScanners(readPt));
-        return keyValueScanners;
-    }
 }


[08/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index 34292ba..fdfd75b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -28,172 +28,119 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
+import java.math.BigDecimal;
 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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
+import org.apache.curator.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.TableNotFoundException;
+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.TestUtil;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
 
+@RunWith(Parameterized.class)
+public class ViewIT extends SplitSystemCatalogIT {
 
-public class ViewIT extends BaseViewIT {
-	
-    public ViewIT(boolean transactional) {
-		super(transactional);
-	}
-
-    @Test
-    public void testReadOnlyOnReadOnlyView() throws Exception {
-        Connection earlierCon = DriverManager.getConnection(getUrl());
-        Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) "+ tableDDLOptions;
-        conn.createStatement().execute(ddl);
-        String fullParentViewName = "V_" + generateUniqueName();
-        ddl = "CREATE VIEW " + fullParentViewName + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
-        conn.createStatement().execute(ddl);
-        try {
-            conn.createStatement().execute("UPSERT INTO " + fullParentViewName + " VALUES(1)");
-            fail();
-        } catch (ReadOnlyTableException e) {
-            
-        }
-        for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ")");
-        }
-        conn.commit();
-        
-        analyzeTable(conn, fullParentViewName, transactional);
-        
-        List<KeyRange> splits = getAllSplits(conn, fullParentViewName);
-        assertEquals(4, splits.size());
-        
-        int count = 0;
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
-        while (rs.next()) {
-            assertEquals(count++, rs.getInt(1));
-        }
-        assertEquals(10, count);
-        
-        count = 0;
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullParentViewName);
-        while (rs.next()) {
-            count++;
-            assertEquals(count + 5, rs.getInt(1));
-        }
-        assertEquals(4, count);
-        count = 0;
-        rs = earlierCon.createStatement().executeQuery("SELECT k FROM " + fullParentViewName);
-        while (rs.next()) {
-            count++;
-            assertEquals(count + 5, rs.getInt(1));
-        }
-        assertEquals(4, count);
-        String fullViewName = "V_" + generateUniqueName();
-        ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullParentViewName + " WHERE k < 9";
-        conn.createStatement().execute(ddl);
-        try {
-            conn.createStatement().execute("UPSERT INTO " + fullViewName + " VALUES(1)");
-            fail();
-        } catch (ReadOnlyTableException e) {
-            
-        } finally {
-            conn.close();
-        }
+    protected String tableDDLOptions;
+    protected boolean transactional;
 
-        conn = DriverManager.getConnection(getUrl());
-        count = 0;
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullViewName);
-        while (rs.next()) {
-            count++;
-            assertEquals(count + 5, rs.getInt(1));
+    public ViewIT(boolean transactional) {
+        StringBuilder optionBuilder = new StringBuilder();
+        this.transactional = transactional;
+        if (transactional) {
+            optionBuilder.append(" TRANSACTIONAL=true ");
         }
-        assertEquals(3, count);
+        this.tableDDLOptions = optionBuilder.toString();
     }
 
-    @Test
-    public void testNonSaltedUpdatableViewWithIndex() throws Exception {
-        testUpdatableViewWithIndex(null, false);
-    }
-    
-    @Test
-    public void testNonSaltedUpdatableViewWithLocalIndex() throws Exception {
-        testUpdatableViewWithIndex(null, true);
+    @Parameters(name = "transactional = {0}")
+    public static Collection<Boolean> data() {
+        return Arrays.asList(new Boolean[] { false, true });
     }
     
-    @Test
-    public void testUpdatableOnUpdatableView() throws Exception {
-        String viewName = testUpdatableView(null);
-        Connection conn = DriverManager.getConnection(getUrl());
-        String fullViewName = "V_" + generateUniqueName();
-        String ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + viewName + " WHERE k3 = 2";
-        conn.createStatement().execute(ddl);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName);
-        assertTrue(rs.next());
-        assertEquals(1, rs.getInt(1));
-        assertEquals(109, rs.getInt(2));
-        assertEquals(2, rs.getInt(3));
-        assertFalse(rs.next());
-
-        conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2) VALUES(122)");
-        conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName + " WHERE k2 >= 120");
-        assertTrue(rs.next());
-        assertEquals(1, rs.getInt(1));
-        assertEquals(122, rs.getInt(2));
-        assertEquals(2, rs.getInt(3));
-        assertFalse(rs.next());
-        
-        try {
-            conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2,k3) VALUES(123,3)");
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN.getErrorCode(), e.getErrorCode());
-        }
-
-        try {
-            conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2,k3) select k2, 3 from " + fullViewName);
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN.getErrorCode(), e.getErrorCode());
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        NUM_SLAVES_BASE = 6;
+        Map<String, String> props = Collections.emptyMap();
+        boolean splitSystemCatalog = (driver == null);
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+        serverProps.put("hbase.coprocessor.abortonerror", "false");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(props.entrySet().iterator()));
+        // Split SYSTEM.CATALOG once after the mini-cluster is started
+        if (splitSystemCatalog) {
+            splitSystemCatalog();
         }
     }
-
+    
     @Test
     public void testReadOnlyOnUpdatableView() throws Exception {
-        String viewName = testUpdatableView(null);
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        String ddl =
+                "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1
+                        + " WHERE k3 > 1 and k3 < 50";
+        testUpdatableView(fullTableName, fullViewName1, fullViewName2, ddl, null, tableDDLOptions);
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullViewName = "V_" + generateUniqueName();
-        String ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + viewName + " WHERE k3 > 1 and k3 < 50";
-        conn.createStatement().execute(ddl);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName);
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2);
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(109, rs.getInt(2));
         assertEquals(2, rs.getInt(3));
         assertFalse(rs.next());
-
         try {
-            conn.createStatement().execute("UPSERT INTO " + fullViewName + " VALUES(1)");
+            conn.createStatement().execute("UPSERT INTO " + fullViewName2 + " VALUES(1)");
             fail();
         } catch (ReadOnlyTableException e) {
             
@@ -201,22 +148,27 @@ public class ViewIT extends BaseViewIT {
         
         conn.createStatement().execute("UPSERT INTO " + fullTableName + "(k1, k2,k3) VALUES(1, 122, 5)");
         conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName + " WHERE k2 >= 120");
+        rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2 + " WHERE k2 >= 120");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(122, rs.getInt(2));
         assertEquals(5, rs.getInt(3));
         assertFalse(rs.next());
     }
-    
+
     @Test
     public void testDisallowDropOfReferencedColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName1 = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName1 + "(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1 + " WHERE v2 != 'foo'";
+        conn.createStatement().execute(ddl);
         
         try {
             conn.createStatement().execute("ALTER VIEW " + fullViewName1 + " DROP COLUMN v1");
@@ -225,10 +177,6 @@ public class ViewIT extends BaseViewIT {
             assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
         }
         
-        String fullViewName2 = "V_" + generateUniqueName();
-        ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1 + " WHERE v2 != 'foo'";
-        conn.createStatement().execute(ddl);
-
         try {
             conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v1");
             fail();
@@ -249,12 +197,16 @@ public class ViewIT extends BaseViewIT {
     public void testReadOnlyViewWithCaseSensitiveTableNames() throws Exception {
         Connection earlierCon = DriverManager.getConnection(getUrl());
         Connection conn = DriverManager.getConnection(getUrl());
+        String schemaName = TestUtil.DEFAULT_SCHEMA_NAME + "_" + generateUniqueName();
         String caseSensitiveTableName = "\"t_" + generateUniqueName() + "\"" ;
-        String ddl = "CREATE TABLE " + caseSensitiveTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
-        conn.createStatement().execute(ddl);
+        String fullTableName = SchemaUtil.getTableName(schemaName, caseSensitiveTableName);
         String caseSensitiveViewName = "\"v_" + generateUniqueName() + "\"" ;
-        ddl = "CREATE VIEW " + caseSensitiveViewName + " (v2 VARCHAR) AS SELECT * FROM " + caseSensitiveTableName + " WHERE k > 5";
+
+        String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW " + caseSensitiveViewName + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        
         try {
             conn.createStatement().execute("UPSERT INTO " + caseSensitiveViewName + " VALUES(1)");
             fail();
@@ -262,7 +214,7 @@ public class ViewIT extends BaseViewIT {
             
         }
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO " + caseSensitiveTableName + " VALUES(" + i + ")");
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ")");
         }
         conn.commit();
         
@@ -285,11 +237,14 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testReadOnlyViewWithCaseSensitiveColumnNames() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         String ddl = "CREATE TABLE " + fullTableName + " (\"k\" INTEGER NOT NULL PRIMARY KEY, \"v1\" INTEGER, \"a\".v2 VARCHAR)" + tableDDLOptions;
+
         conn.createStatement().execute(ddl);
-        String viewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + viewName + " (v VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE \"k\" > 5 and \"v1\" > 1";
         conn.createStatement().execute(ddl);
+        
         try {
             conn.createStatement().execute("UPSERT INTO " + viewName + " VALUES(1)");
             fail();
@@ -313,11 +268,14 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewWithCurrentDate() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+
         String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String viewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + viewName + " (v VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v2 > CURRENT_DATE()-5 AND v2 > DATE '2010-01-01'";
         conn.createStatement().execute(ddl);
+        
         try {
             conn.createStatement().execute("UPSERT INTO " + viewName + " VALUES(1)");
             fail();
@@ -354,27 +312,33 @@ public class ViewIT extends BaseViewIT {
         Properties props = new Properties();
         props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
         Connection conn = DriverManager.getConnection(getUrl(),props);
-        String schemaName1 = "S_" + generateUniqueName();
+        String tableName = "T_" + generateUniqueName();
+        String schemaName1 = SCHEMA1;
         String fullTableName1 = SchemaUtil.getTableName(schemaName1, tableName);
+        String viewName1 = "V_" + generateUniqueName();
+        String viewSchemaName = SCHEMA2;
+        String fullViewName1 = SchemaUtil.getTableName(viewSchemaName, viewName1);
+        String viewName2 = "V_" + generateUniqueName();
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, viewName1);
+        
         if (isNamespaceMapped) {
             conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName1);
         }
-		String ddl = "CREATE TABLE " + fullTableName1 + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        String ddl = "CREATE TABLE " + fullTableName1 + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
         conn.createStatement().execute(ddl);
         assertTrue(admin.tableExists(SchemaUtil.getPhysicalTableName(SchemaUtil.normalizeIdentifier(fullTableName1),
                 conn.unwrap(PhoenixConnection.class).getQueryServices().getProps())));
-        String viewName = "V_" + generateUniqueName();
-        String viewSchemaName = "S_" + generateUniqueName();
-        String fullViewName1 = SchemaUtil.getTableName(viewSchemaName, viewName);
+        
         ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
         conn.createStatement().execute(ddl);
-        String fullViewName2 = "V_" + generateUniqueName();
+        
         ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
         conn.createStatement().execute(ddl);
+        
         conn.createStatement().executeQuery("SELECT * FROM " + fullViewName1);
         conn.createStatement().executeQuery("SELECT * FROM " + fullViewName2);
-        ddl = "DROP VIEW " + viewName;
+        ddl = "DROP VIEW " + viewName1;
         try {
             conn.createStatement().execute(ddl);
             fail();
@@ -389,20 +353,20 @@ public class ViewIT extends BaseViewIT {
         } catch (TableNotFoundException ignore) {
         }
         ddl = "DROP TABLE " + fullTableName1;
-        validateCannotDropTableWithChildViewsWithoutCascade(conn, fullTableName1);
         ddl = "DROP VIEW " + fullViewName2;
         conn.createStatement().execute(ddl);
         ddl = "DROP TABLE " + fullTableName1;
         conn.createStatement().execute(ddl);
     }
 
-    
+
     @Test
     public void testDisallowDropOfColumnOnParentTable() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String viewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         
@@ -416,46 +380,171 @@ public class ViewIT extends BaseViewIT {
    
     @Test
     public void testViewAndTableAndDropCascade() throws Exception {
-    	// Setup
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
-        conn.createStatement().execute(ddl);
-        String viewName = "V_" + generateUniqueName();
-        String viewSchemaName = "S_" + generateUniqueName();
-        String fullViewName1 = SchemaUtil.getTableName(viewSchemaName, viewName);
-        ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        
+        String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE LOCAL INDEX idx on " + fullViewName1 + "(v2)";
+        String indexName = generateUniqueName();
+        ddl = "CREATE LOCAL INDEX " + indexName + " on " + fullViewName1 + "(v2)";
         conn.createStatement().execute(ddl);
-        String fullViewName2 = SchemaUtil.getTableName(viewSchemaName, "V_" + generateUniqueName());
         ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
         conn.createStatement().execute(ddl);
+        
+        // dropping base table without cascade should fail
+        try {
+            conn.createStatement().execute("DROP TABLE " + fullTableName );
+            fail();
+        }
+        catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+        }
 
-        validateCannotDropTableWithChildViewsWithoutCascade(conn, fullTableName);
+        // drop table cascade should succeed
+        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
         
-        // Execute DROP...CASCADE
+        validateViewDoesNotExist(conn, fullViewName1);
+        validateViewDoesNotExist(conn, fullViewName2);
+
+    }
+    
+    @Test
+    public void testRecreateDroppedTableWithChildViews() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        
+        String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        String indexName = generateUniqueName();
+        ddl = "CREATE INDEX " + indexName + " on " + fullViewName1 + "(v2)";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
+        conn.createStatement().execute(ddl);
+        
+        // drop table cascade should succeed
         conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
         
         validateViewDoesNotExist(conn, fullViewName1);
         validateViewDoesNotExist(conn, fullViewName2);
+
+        // recreate the table that was dropped
+        conn.createStatement().execute(tableDdl);
+        // the two child views should still not exist
+        try {
+            PhoenixRuntime.getTableNoCache(conn, fullViewName1);
+            fail();
+        } catch (SQLException e) {
+        }
+        try {
+            PhoenixRuntime.getTableNoCache(conn, fullViewName2);
+            fail();
+        } catch (SQLException e) {
+        }
     }
     
     @Test
-    public void testViewAndTableAndDropCascadeWithIndexes() throws Exception {
+    public void testRecreateIndexWhoseAncestorWasDropped() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullTableName2 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         
-    	// Setup - Tables and Views with Indexes
-    	Connection conn = DriverManager.getConnection(getUrl());
-		if (tableDDLOptions.length()!=0)
-			tableDDLOptions+=",";
-		tableDDLOptions+="IMMUTABLE_ROWS=true";
+        String tableDdl = "CREATE TABLE " + fullTableName1 + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        tableDdl = "CREATE TABLE " + fullTableName2 + "  (k INTEGER NOT NULL PRIMARY KEY, v3 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        String indexName = generateUniqueName();
+        ddl = "CREATE INDEX " + indexName + " on " + fullViewName1 + "(v2)";
+        conn.createStatement().execute(ddl);
+        try {
+                // this should fail because an index with this name is present
+            ddl = "CREATE INDEX " + indexName + " on " + fullTableName2 + "(v1)";
+            conn.createStatement().execute(ddl);
+            fail();
+        }
+        catch(SQLException e) {
+        }
+        
+        // drop table cascade should succeed
+        conn.createStatement().execute("DROP TABLE " + fullTableName1 + " CASCADE");
+        
+        // should be able to reuse the index name 
+        ddl = "CREATE INDEX " + indexName + " on " + fullTableName2 + "(v3)";
+        conn.createStatement().execute(ddl);
+        
+        String fullIndexName = SchemaUtil.getTableName(SCHEMA2, indexName);
+        PTable index = PhoenixRuntime.getTableNoCache(conn, fullIndexName);
+        // the index should have v3 but not v2
+        validateCols(index);
+    }
+    
+    @Test
+    public void testRecreateViewWhoseParentWasDropped() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullTableName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        
+        String tableDdl = "CREATE TABLE " + fullTableName1 + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        tableDdl = "CREATE TABLE " + fullTableName2 + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        
+        // drop table cascade should succeed
+        conn.createStatement().execute("DROP TABLE " + fullTableName1 + " CASCADE");
+        
+        // should be able to reuse the view name 
+        ddl = "CREATE VIEW " + fullViewName1 + " (v3 VARCHAR) AS SELECT * FROM " + fullTableName2 + " WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        
+        PTable view = PhoenixRuntime.getTableNoCache(conn, fullViewName1);
+        // the view should have v3 but not v2
+        validateCols(view);
+    }
+
+    private void validateCols(PTable table) {
+        final String prefix = table.getType() == PTableType.INDEX ? "0:" : "";
+        Predicate<PColumn> predicate = new Predicate<PColumn>() {
+            @Override
+            public boolean apply(PColumn col) {
+                return col.getName().getString().equals(prefix + "V3")
+                        || col.getName().getString().equals(prefix + "V2");
+            }
+        };
+        List<PColumn> colList = table.getColumns();
+        Collection<PColumn> filteredCols = Collections2.filter(colList, predicate);
+        assertEquals(1, filteredCols.size());
+        assertEquals(prefix + "V3", filteredCols.iterator().next().getName().getString());
+    }
+    
+    @Test
+    public void testViewAndTableAndDropCascadeWithIndexes() throws Exception {
+        // Setup - Tables and Views with Indexes
+        Connection conn = DriverManager.getConnection(getUrl());
+        if (tableDDLOptions.length()!=0)
+            tableDDLOptions+=",";
+        tableDDLOptions+="IMMUTABLE_ROWS=true";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
         String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String viewSchemaName = "S_" + generateUniqueName();
-        String fullViewName1 = SchemaUtil.getTableName(viewSchemaName, "V_" + generateUniqueName());
-        String fullViewName2 = SchemaUtil.getTableName(viewSchemaName, "V_" + generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
         String indexName1 = "I_" + generateUniqueName();
         String indexName2 = "I_" + generateUniqueName();
         String indexName3 = "I_" + generateUniqueName();
+        
         ddl = "CREATE INDEX " + indexName1 + " ON " + fullTableName + " (v1)";
         conn.createStatement().execute(ddl);
         ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
@@ -466,9 +555,8 @@ public class ViewIT extends BaseViewIT {
         conn.createStatement().execute(ddl);
         ddl = "CREATE INDEX " + indexName3 + " ON " + fullViewName2 + " (v2)";
         conn.createStatement().execute(ddl);
-
-        validateCannotDropTableWithChildViewsWithoutCascade(conn, fullTableName);
         
+
         // Execute DROP...CASCADE
         conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
         
@@ -478,28 +566,16 @@ public class ViewIT extends BaseViewIT {
     }
 
 
-	private void validateCannotDropTableWithChildViewsWithoutCascade(Connection conn, String tableName) throws SQLException {
-		String ddl;
-		try {
-	        ddl = "DROP TABLE " + tableName;
-	        conn.createStatement().execute(ddl);
-	        fail("Should not be able to drop table " + tableName + " with child views without explictly specifying CASCADE");
-        }  catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-        }
-	}
-
-
-	private void validateViewDoesNotExist(Connection conn, String fullViewName)	throws SQLException {
-		try {
-        	String ddl1 = "DROP VIEW " + fullViewName;
+    private void validateViewDoesNotExist(Connection conn, String fullViewName)    throws SQLException {
+        try {
+            String ddl1 = "DROP VIEW " + fullViewName;
             conn.createStatement().execute(ddl1);
             fail("View " + fullViewName + " should have been deleted when parent was dropped");
         } catch (TableNotFoundException e) {
-        	//Expected
+            //Expected
         }
-	}
-	
+    }
+    
     @Test
     public void testViewUsesTableGlobalIndex() throws Exception {
         testViewUsesTableIndex(false);
@@ -515,17 +591,17 @@ public class ViewIT extends BaseViewIT {
         ResultSet rs;
         // Use unique name for table with local index as otherwise we run into issues
         // when we attempt to drop the table (with the drop metadata option set to false
-        String fullTableName = this.fullTableName + (localIndex ? "_WITH_LI" : "_WITHOUT_LI");
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName()) + (localIndex ? "_WITH_LI" : "_WITHOUT_LI");
         Connection conn = DriverManager.getConnection(getUrl());
         String ddl = "CREATE TABLE " + fullTableName + "  (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
         String indexName1 = "I_" + generateUniqueName();
-        String fullIndexName1 = SchemaUtil.getTableName(schemaName, indexName1);
+        String fullIndexName1 = SchemaUtil.getTableName(SCHEMA1, indexName1);
         conn.createStatement().execute("CREATE " + (localIndex ? "LOCAL " : "") + " INDEX " + indexName1 + " ON " + fullTableName + "(k3, k2) INCLUDE(s1, s2)");
         String indexName2 = "I_" + generateUniqueName();
         conn.createStatement().execute("CREATE INDEX " + indexName2 + " ON " + fullTableName + "(k3, k2, s2)");
         
-        String fullViewName = "V_" + generateUniqueName();
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName + " WHERE s1 = 'foo'";
         conn.createStatement().execute(ddl);
         String[] s1Values = {"foo","bar"};
@@ -564,15 +640,20 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testCreateViewDefinesPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName + "(v2 VARCHAR, k3 VARCHAR PRIMARY KEY) AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
         conn.createStatement().execute(ddl);
 
         // assert PK metadata
-        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, fullViewName);
-        assertPKs(rs, new String[] {"K1", "K2", "K3"});
+        ResultSet rs =
+                conn.getMetaData().getPrimaryKeys(null,
+                    SchemaUtil.getSchemaNameFromFullName(fullViewName),
+                    SchemaUtil.getTableNameFromFullName(fullViewName));
+        assertPKs(rs, new String[] {"K3"});
         
         // sanity check upserts into base table and view
         conn.createStatement().executeUpdate("upsert into " + fullTableName + " (k1, k2, v1) values (1, 1, 1)");
@@ -593,43 +674,54 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testCreateViewDefinesPKConstraint() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName + "(v2 VARCHAR, k3 VARCHAR, k4 INTEGER NOT NULL, CONSTRAINT PKVEW PRIMARY KEY (k3, k4)) AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
         conn.createStatement().execute(ddl);
-
+        
         // assert PK metadata
-        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, fullViewName);
-        assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
+        ResultSet rs =
+                conn.getMetaData().getPrimaryKeys(null,
+                    SchemaUtil.getSchemaNameFromFullName(fullViewName),
+                    SchemaUtil.getTableNameFromFullName(fullViewName));
+        assertPKs(rs, new String[] {"K3", "K4"});
     }
     
     @Test
     public void testViewAddsPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName2 = fullTableName;
-		String ddl = "CREATE TABLE " + fullTableName2 + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = generateUniqueName();
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, viewName);
+        
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
+        
         ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
         conn.createStatement().execute(ddl);
 
         // assert PK metadata
-        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, fullViewName);
-        assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
+        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, SCHEMA2, viewName);
+        assertPKs(rs, new String[] {"K3", "K4"});
     }
     
     @Test
     public void testViewAddsPKColumnWhoseParentsLastPKIsVarLength() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 VARCHAR NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName1 = "V_" + generateUniqueName();
-        ddl = "CREATE VIEW " + fullViewName1 + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
-        ddl = "ALTER VIEW " + fullViewName1 + " ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
+        
+        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
         try {
             conn.createStatement().execute(ddl);
             fail("View cannot extend PK if parent's last PK is variable length. See https://issues.apache.org/jira/browse/PHOENIX-978.");
@@ -639,7 +731,7 @@ public class ViewIT extends BaseViewIT {
         String fullViewName2 = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName2 + " (k3 VARCHAR PRIMARY KEY)  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         try {
-        	conn.createStatement().execute(ddl);
+            conn.createStatement().execute(ddl);
         } catch (SQLException e) {
             assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(), e.getErrorCode());
         }
@@ -648,23 +740,28 @@ public class ViewIT extends BaseViewIT {
     @Test(expected=ColumnAlreadyExistsException.class)
     public void testViewAddsClashingPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
+        
         ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, k2 VARCHAR PRIMARY KEY, v2 INTEGER";
         conn.createStatement().execute(ddl);
     }
-    
+
     @Test
     public void testViewAddsNotNullPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        String fullViewName = "V_" + generateUniqueName();
         ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
+        
         try {
             ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR NOT NULL PRIMARY KEY"; 
             conn.createStatement().execute(ddl);
@@ -677,15 +774,17 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testQueryViewStatementOptimization() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
         String sql = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(sql);
-        String fullViewName1 = "V_" + generateUniqueName();
         sql = "CREATE VIEW " + fullViewName1 + "  AS SELECT * FROM " + fullTableName;
         conn.createStatement().execute(sql);
-        String fullViewName2 = "V_" + generateUniqueName();
         sql = "CREATE VIEW " + fullViewName2 + "  AS SELECT * FROM " + fullTableName + " WHERE k1 = 1.0";
         conn.createStatement().execute(sql);
         
+        
         sql = "SELECT * FROM " + fullViewName1 + " order by k1, k2";
         PreparedStatement stmt = conn.prepareStatement(sql);
         QueryPlan plan = PhoenixRuntime.getOptimizedQueryPlan(stmt);
@@ -702,13 +801,14 @@ public class ViewIT extends BaseViewIT {
       Properties props = new Properties();
       Connection conn1 = DriverManager.getConnection(getUrl(), props);
       conn1.setAutoCommit(true);
-      String tableName=generateUniqueName();
-      String viewName=generateUniqueName();
+      String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+      String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
       conn1.createStatement().execute(
         "CREATE TABLE "+tableName+" (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) UPDATE_CACHE_FREQUENCY=1000000");
       conn1.createStatement().execute("upsert into "+tableName+" values ('row1', 'value1', 'key1')");
       conn1.createStatement().execute(
         "CREATE VIEW "+viewName+" (v43 VARCHAR) AS SELECT * FROM "+tableName+" WHERE v1 = 'value1'");
+      
       ResultSet rs = conn1.createStatement()
           .executeQuery("SELECT * FROM "+tableName+" WHERE v1 = 'value1'");
       assertTrue(rs.next());
@@ -724,60 +824,59 @@ public class ViewIT extends BaseViewIT {
         props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());
 
         try (Connection conn = DriverManager.getConnection(getUrl(), props);
-            HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+                HBaseAdmin admin =
+                        conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
 
             conn.createStatement().execute("CREATE SCHEMA " + NS);
 
             // test for a view that is in non-default schema
-            {
-                HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(NS, TBL));
-                desc.addFamily(new HColumnDescriptor(CF));
-                admin.createTable(desc);
+            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(NS, TBL));
+            desc.addFamily(new HColumnDescriptor(CF));
+            admin.createTable(desc);
 
-                String view = NS + "." + TBL;
-                conn.createStatement().execute(
-                    "CREATE VIEW " + view + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
+            String view1 = NS + "." + TBL;
+            conn.createStatement().execute(
+                "CREATE VIEW " + view1 + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
 
-                assertTrue(QueryUtil.getExplainPlan(
-                    conn.createStatement().executeQuery("explain select * from " + view))
+            assertTrue(QueryUtil
+                    .getExplainPlan(
+                        conn.createStatement().executeQuery("explain select * from " + view1))
                     .contains(NS + ":" + TBL));
 
-                conn.createStatement().execute("DROP VIEW " + view);
-            }
+            
 
-            // test for a view whose name contains a dot (e.g. "AAA.BBB") in default schema (for backward compatibility)
-            {
-                HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(NS + "." + TBL));
-                desc.addFamily(new HColumnDescriptor(CF));
-                admin.createTable(desc);
+            // test for a view whose name contains a dot (e.g. "AAA.BBB") in default schema (for
+            // backward compatibility)
+            desc = new HTableDescriptor(TableName.valueOf(NS + "." + TBL));
+            desc.addFamily(new HColumnDescriptor(CF));
+            admin.createTable(desc);
 
-                String view = "\"" + NS + "." + TBL + "\"";
-                conn.createStatement().execute(
-                    "CREATE VIEW " + view + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
+            String view2 = "\"" + NS + "." + TBL + "\"";
+            conn.createStatement().execute(
+                "CREATE VIEW " + view2 + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
 
-                assertTrue(QueryUtil.getExplainPlan(
-                    conn.createStatement().executeQuery("explain select * from " + view))
+            assertTrue(QueryUtil
+                    .getExplainPlan(
+                        conn.createStatement().executeQuery("explain select * from " + view2))
                     .contains(NS + "." + TBL));
 
-                conn.createStatement().execute("DROP VIEW " + view);
-            }
-
             // test for a view whose name contains a dot (e.g. "AAA.BBB") in non-default schema
-            {
-                HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(NS, NS + "." + TBL));
-                desc.addFamily(new HColumnDescriptor(CF));
-                admin.createTable(desc);
+            desc = new HTableDescriptor(TableName.valueOf(NS, NS + "." + TBL));
+            desc.addFamily(new HColumnDescriptor(CF));
+            admin.createTable(desc);
 
-                String view = NS + ".\"" + NS + "." + TBL + "\"";
-                conn.createStatement().execute(
-                    "CREATE VIEW " + view + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
+            String view3 = NS + ".\"" + NS + "." + TBL + "\"";
+            conn.createStatement().execute(
+                "CREATE VIEW " + view3 + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
 
-                assertTrue(QueryUtil.getExplainPlan(
-                    conn.createStatement().executeQuery("explain select * from " + view))
+            assertTrue(QueryUtil
+                    .getExplainPlan(
+                        conn.createStatement().executeQuery("explain select * from " + view3))
                     .contains(NS + ":" + NS + "." + TBL));
-
-                conn.createStatement().execute("DROP VIEW " + view);
-            }
+            
+            conn.createStatement().execute("DROP VIEW " + view1);
+            conn.createStatement().execute("DROP VIEW " + view2);
+            conn.createStatement().execute("DROP VIEW " + view3);
 
             conn.createStatement().execute("DROP SCHEMA " + NS);
         }
@@ -793,14 +892,14 @@ public class ViewIT extends BaseViewIT {
     }
 
     @Test
-    public void testCompositeDescPK() throws SQLException {
+    public void testCompositeDescPK() throws Exception {
         Properties props = new Properties();
         try (Connection globalConn = DriverManager.getConnection(getUrl(), props)) {
-            String tableName = generateUniqueName();
-            String viewName1 = generateUniqueName();
-            String viewName2 = generateUniqueName();
-            String viewName3 = generateUniqueName();
-            String viewName4 = generateUniqueName();
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String viewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            String viewName3 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewName4 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
 
             // create global base table
             globalConn.createStatement().execute("CREATE TABLE " + tableName
@@ -823,6 +922,17 @@ public class ViewIT extends BaseViewIT {
                                 + " (pk1 VARCHAR(10) NOT NULL, pk2 VARCHAR(10) NOT NULL, col1 DATE, col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC)) AS SELECT * FROM "
                                 + tableName + " WHERE KEY_PREFIX = 'abc' ");
 
+                // view with composite PK with multiple Date PK values DESC
+                tenantConn.createStatement()
+                        .execute("CREATE VIEW " + viewName3
+                                + " (pk1 DATE(10) NOT NULL, pk2 DATE(10) NOT NULL, col1 VARCHAR(10), col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC)) AS SELECT * FROM "
+                                + tableName + " WHERE KEY_PREFIX = 'ab3' ");
+                
+                tenantConn.createStatement()
+                .execute("CREATE VIEW " + viewName4
+                        + " (pk1 DATE(10) NOT NULL, pk2 DECIMAL NOT NULL, pk3 VARCHAR(10) NOT NULL, col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC, pk3 DESC)) AS SELECT * FROM "
+                        + tableName + " WHERE KEY_PREFIX = 'ab4' ");
+
                 // upsert rows
                 upsertRows(viewName1, tenantConn);
                 upsertRows(viewName2, tenantConn);
@@ -835,12 +945,6 @@ public class ViewIT extends BaseViewIT {
                 validate(viewName1, tenantConn, whereClauses, expectedArray);
                 validate(viewName2, tenantConn, whereClauses, expectedArray);
 
-                // view with composite PK with multiple Date PK values DESC
-                tenantConn.createStatement()
-                        .execute("CREATE VIEW " + viewName3
-                                + " (pk1 DATE(10) NOT NULL, pk2 DATE(10) NOT NULL, col1 VARCHAR(10), col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC)) AS SELECT * FROM "
-                                + tableName + " WHERE KEY_PREFIX = 'ab3' ");
-
                 tenantConn.createStatement().execute("UPSERT INTO " + viewName3
                         + " (pk1, pk2, col1, col3) VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), TO_DATE('2017-10-16 21:00:00', 'yyyy-MM-dd HH:mm:ss'), 'txt1', 10)");
                 tenantConn.createStatement().execute("UPSERT INTO " + viewName3
@@ -862,11 +966,6 @@ public class ViewIT extends BaseViewIT {
                                 "pk1 < TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')" };
                 validate(viewName3, tenantConn, view3WhereClauses, expectedArray);
 
-                tenantConn.createStatement()
-                        .execute("CREATE VIEW " + viewName4
-                                + " (pk1 DATE(10) NOT NULL, pk2 DECIMAL NOT NULL, pk3 VARCHAR(10) NOT NULL, col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC, pk3 DESC)) AS SELECT * FROM "
-                                + tableName + " WHERE KEY_PREFIX = 'ab4' ");
-
                 tenantConn.createStatement().execute("UPSERT INTO " + viewName4
                         + " (pk1, pk2, pk3, col3) VALUES (TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 1, 'txt1', 10)");
                 tenantConn.createStatement().execute("UPSERT INTO " + viewName4
@@ -901,8 +1000,8 @@ public class ViewIT extends BaseViewIT {
                 Connection conn2 = DriverManager.getConnection(getUrl());
                 Statement s = conn.createStatement();
                 Statement s2 = conn2.createStatement()) {
-            String tableName = generateUniqueName();
-            String viewName = generateUniqueName();
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
             String indexName = generateUniqueName();
             helpTestQueryForViewOnTableThatHasIndex(s, s2, tableName, viewName, indexName);
         }
@@ -912,8 +1011,8 @@ public class ViewIT extends BaseViewIT {
     public void testQueryForViewOnTableThatHasIndex() throws SQLException {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement s = conn.createStatement()) {
-            String tableName = generateUniqueName();
-            String viewName = generateUniqueName();
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
             String indexName = generateUniqueName();
             helpTestQueryForViewOnTableThatHasIndex(s, s, tableName, viewName, indexName);
         }
@@ -966,4 +1065,237 @@ public class ViewIT extends BaseViewIT {
                 + " (pk1, pk2, col1, col3) VALUES ('testb', 'testa', TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss'), 10)");
         tenantConn.commit();
     }
+    
+    @Test
+    public void testUpdatingPropertyOnBaseTable() throws Exception {
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement()
+                    .execute("create table " + fullTableName
+                            + "(tenantId CHAR(15) NOT NULL, pk1 integer NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
+                            + "(tenantId, pk1)) MULTI_TENANT=true");
+            conn.createStatement().execute("CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
+            
+            conn.createStatement()
+                    .execute("ALTER TABLE " + fullTableName + " set IMMUTABLE_ROWS = true");
+            
+            // fetch the latest tables
+            PTable table = PhoenixRuntime.getTableNoCache(conn, fullTableName);
+            PTable view = PhoenixRuntime.getTableNoCache(conn, fullViewName);
+            assertEquals("IMMUTABLE_ROWS property set incorrectly", true, table.isImmutableRows());
+            assertEquals("IMMUTABLE_ROWS property set incorrectly", true, view.isImmutableRows());
+        }
+    }
+
+    public static String testUpdatableView(String fullTableName, String fullViewName, String fullChildViewName,
+            String childViewDDL, Integer saltBuckets, String tableDDLOptions) throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        if (saltBuckets != null) {
+            if (tableDDLOptions.length() != 0)
+                tableDDLOptions += ",";
+            tableDDLOptions += (" SALT_BUCKETS=" + saltBuckets);
+        }
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))"
+                + tableDDLOptions;
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName + " WHERE k1 = 1";
+        conn.createStatement().execute(ddl);
+        ArrayList<String> splitPoints = Lists.newArrayList(fullTableName, fullViewName);
+        if (fullChildViewName != null) {
+            conn.createStatement().execute(childViewDDL);
+            splitPoints.add(fullChildViewName);
+        }
+
+        for (int i = 0; i < 10; i++) {
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + (i % 4) + "," + (i + 100) + ","
+                    + (i > 5 ? 2 : 1) + ")");
+        }
+        conn.commit();
+        ResultSet rs;
+
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
+        assertTrue(rs.next());
+        assertEquals(10, rs.getInt(1));
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + fullViewName);
+        assertTrue(rs.next());
+        assertEquals(3, rs.getInt(1));
+        rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(101, rs.getInt(2));
+        assertEquals(1, rs.getInt(3));
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(105, rs.getInt(2));
+        assertEquals(1, rs.getInt(3));
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(109, rs.getInt(2));
+        assertEquals(2, rs.getInt(3));
+        assertFalse(rs.next());
+
+        conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2,S,k3) VALUES(120,'foo',50.0)");
+        conn.createStatement().execute("UPSERT INTO " + fullViewName + "(k2,S,k3) VALUES(121,'bar',51.0)");
+        conn.commit();
+        rs = conn.createStatement().executeQuery("SELECT k1, k2 FROM " + fullViewName + " WHERE k2 >= 120");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(120, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(121, rs.getInt(2));
+        assertFalse(rs.next());
+        conn.close();
+        return fullViewName;
+    }
+
+    public static Pair<String, Scan> testUpdatableViewIndex(String fullTableName, Integer saltBuckets,
+            boolean localIndex, String viewName) throws Exception {
+        ResultSet rs;
+        Connection conn = DriverManager.getConnection(getUrl());
+        String viewIndexName1 = "I_" + generateUniqueName();
+        String viewIndexPhysicalName = MetaDataUtil.getViewIndexPhysicalName(fullTableName);
+        if (localIndex) {
+            conn.createStatement().execute("CREATE LOCAL INDEX " + viewIndexName1 + " on " + viewName + "(k3)");
+        } else {
+            conn.createStatement().execute("CREATE INDEX " + viewIndexName1 + " on " + viewName + "(k3) include (s)");
+        }
+        conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,S,k3) VALUES(120,'foo',50.0)");
+        conn.commit();
+
+        analyzeTable(conn, viewName);
+        List<KeyRange> splits = getAllSplits(conn, viewIndexName1);
+        // More guideposts with salted, since it's already pre-split at salt
+        // buckets
+        assertEquals(saltBuckets == null ? 6 : 8, splits.size());
+
+        String query = "SELECT k1, k2, k3, s FROM " + viewName + " WHERE k3 = 51.0";
+        rs = conn.createStatement().executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(121, rs.getInt(2));
+        assertTrue(BigDecimal.valueOf(51.0).compareTo(rs.getBigDecimal(3)) == 0);
+        assertEquals("bar", rs.getString(4));
+        assertFalse(rs.next());
+        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+        String queryPlan = QueryUtil.getExplainPlan(rs);
+        if (localIndex) {
+            assertEquals("CLIENT PARALLEL " + (saltBuckets == null ? 1 : saltBuckets) + "-WAY RANGE SCAN OVER "
+                    + fullTableName + " [1,51]\n" + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
+                    queryPlan);
+        } else {
+            assertEquals(saltBuckets == null
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [" + Short.MIN_VALUE + ",51]"
+                    : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0,"
+                            + Short.MIN_VALUE + ",51] - [" + (saltBuckets.intValue() - 1) + "," + Short.MIN_VALUE
+                            + ",51]\nCLIENT MERGE SORT",
+                    queryPlan);
+        }
+
+        String viewIndexName2 = "I_" + generateUniqueName();
+        if (localIndex) {
+            conn.createStatement().execute("CREATE LOCAL INDEX " + viewIndexName2 + " on " + viewName + "(s)");
+        } else {
+            conn.createStatement().execute("CREATE INDEX " + viewIndexName2 + " on " + viewName + "(s)");
+        }
+
+        // new index hasn't been analyzed yet
+        splits = getAllSplits(conn, viewIndexName2);
+        assertEquals(saltBuckets == null ? 1 : 3, splits.size());
+
+        // analyze table should analyze all view data
+        analyzeTable(conn, fullTableName);
+        splits = getAllSplits(conn, viewIndexName2);
+        assertEquals(saltBuckets == null ? 6 : 8, splits.size());
+
+        query = "SELECT k1, k2, s FROM " + viewName + " WHERE s = 'foo'";
+        Statement statement = conn.createStatement();
+        rs = statement.executeQuery(query);
+        Scan scan = statement.unwrap(PhoenixStatement.class).getQueryPlan().getContext().getScan();
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(120, rs.getInt(2));
+        assertEquals("foo", rs.getString(3));
+        assertFalse(rs.next());
+        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+        String physicalTableName;
+        if (localIndex) {
+            physicalTableName = fullTableName;
+            assertEquals("CLIENT PARALLEL " + (saltBuckets == null ? 1 : saltBuckets) + "-WAY RANGE SCAN OVER "
+                    + fullTableName + " [" + (2) + ",'foo']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                    + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+        } else {
+            physicalTableName = viewIndexPhysicalName;
+            assertEquals(
+                    saltBuckets == null
+                            ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " ["
+                                    + (Short.MIN_VALUE + 1) + ",'foo']\n" + "    SERVER FILTER BY FIRST KEY ONLY"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName
+                                    + " [0," + (Short.MIN_VALUE + 1) + ",'foo'] - [" + (saltBuckets.intValue() - 1)
+                                    + "," + (Short.MIN_VALUE + 1) + ",'foo']\n"
+                                    + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
+                    QueryUtil.getExplainPlan(rs));
+        }
+        conn.close();
+        return new Pair<>(physicalTableName, scan);
+    }
+    
+    @Test
+    public void testChildViewCreationFails() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, FAILED_VIEWNAME);
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        
+        String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+        conn.createStatement().execute(tableDdl);
+        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        }
+        catch (PhoenixIOException e){
+        }
+        ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
+        conn.createStatement().execute(ddl);
+        
+        // the first child view should not exist
+        try {
+            PhoenixRuntime.getTableNoCache(conn, fullViewName1);
+            fail();
+        } catch (SQLException e) {
+        }
+        
+        // we should be able to load the table
+        PhoenixRuntime.getTableNoCache(conn, fullTableName);
+        // we should be able to load the second view
+        PhoenixRuntime.getTableNoCache(conn, fullViewName2);
+    }
+    
+    private static final String FAILED_VIEWNAME = "FAILED_VIEW";
+    private static final byte[] ROWKEY_TO_FAIL_BYTES = SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2),
+            Bytes.toBytes(FAILED_VIEWNAME));
+    
+    public static class FailingRegionObserver extends SimpleRegionObserver {
+        @Override
+        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+                MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+            if (shouldFail(c, miniBatchOp.getOperation(0))) {
+                // throwing anything other than instances of IOException result
+                // in this coprocessor being unloaded
+                // DoNotRetryIOException tells HBase not to retry this mutation
+                // multiple times
+                throw new DoNotRetryIOException();
+            }
+        }
+
+        private boolean shouldFail(ObserverContext<RegionCoprocessorEnvironment> c, Mutation m) {
+            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
+            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
+                    && (Bytes.equals(ROWKEY_TO_FAIL_BYTES, m.getRow()));
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
index f914256..d86037c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -124,6 +125,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IND_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(false);
@@ -140,12 +142,12 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
                 assertEquals(
-                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1]\n" +
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\n" +
                                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                                 "CLIENT MERGE SORT",
                                 QueryUtil.getExplainPlan(rs));
             } else {
-                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + indexName + "\n"
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
 
@@ -168,7 +170,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
 
-            query = "SELECT char_col1, int_col1 from "+indexName;
+            query = "SELECT char_col1, int_col1 from "+fullIndexName;
             try{
                 rs = conn.createStatement().executeQuery(query);
                 fail();
@@ -500,7 +502,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
             Statement stmt = conn.createStatement();
             stmt.execute(ddl);
 
-            query = "SELECT * FROM " + tableName;
+            query = "SELECT * FROM " + fullTableName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
 
@@ -772,10 +774,25 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute(ddl);
 
             stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-            stmt.setString(1, "a");
-            stmt.setString(2, "y");
+            if (mutable) {
+	            stmt.setString(1, "a");
+	            stmt.setString(2, "y");
+	            stmt.execute();
+	            conn.commit();
+            }
+            stmt.setString(1, "b");
+            stmt.setString(2, "x");
             stmt.execute();
             conn.commit();
+	            
+            // the index table is one row
+            HTable table = new HTable(getUtility().getConfiguration(), fullTableName);
+            ResultScanner resultScanner = table.getScanner(new Scan());
+            for (Result result : resultScanner) {
+            	System.out.println(result);
+            }
+            resultScanner.close();
+            table.close();
 
             query = "SELECT * FROM " + fullTableName;
 
@@ -783,10 +800,15 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("a", rs.getString(1));
-            assertEquals("y", rs.getString(2));
+            assertEquals(mutable ? "y" : "x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("x", rs.getString(2));
+            assertNull(rs.getString(3));
             assertFalse(rs.next());
+            }
         }
-    }
 
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
@@ -1069,6 +1091,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IND_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexeName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         // Check system tables priorities.
         try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin(); 
                 Connection c = DriverManager.getConnection(getUrl())) {
@@ -1107,7 +1130,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
 
             if (!localIndex && mutable) {
                 HTableDescriptor indexTable = admin.getTableDescriptor(
-                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                        org.apache.hadoop.hbase.TableName.valueOf(fullIndexeName));
                 val = indexTable.getValue("PRIORITY");
                 assertNotNull("PRIORITY is not set for table:" + indexTable, val);
                 assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
@@ -1140,7 +1163,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
                 ddl += "(p1 desc, p2))";
             }
             stmt.executeUpdate(ddl);
-            ddl = "CREATE "+ (localIndex ? "LOCAL " : "") + " INDEX " + fullIndexName + " on " + fullTableName + "(a)";
+            ddl = "CREATE "+ (localIndex ? "LOCAL " : "") + " INDEX " + indexName + " on " + fullTableName + "(a)";
             stmt.executeUpdate(ddl);
 
             // upsert a single row

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index 6c8f9e8..f3ee2cb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -29,7 +29,11 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
 
@@ -111,7 +115,8 @@ public class ChildViewsUseParentViewIndexIT extends ParallelStatsDisabledIT {
             // create child of parent view that should be able to use the parent's index
             String childViewDdl = "CREATE VIEW " + childViewName1 + " AS SELECT * FROM " + parentViewName + " WHERE A2 = 'Y'";
             conn.createStatement().execute(childViewDdl);
-            
+
+            PTable childViewPTable = PhoenixRuntime.getTableNoCache(conn, childViewName1);
             // create child of parent view that should *not* be able to use the parent's index
             String grandChildViewDdl1 = "CREATE VIEW " + childViewName2 + " AS SELECT * FROM " + childViewName1 + " WHERE A3 = 'Z'";
             conn.createStatement().execute(grandChildViewDdl1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
index 28aa1e9..e19284a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.end2end.index;
 
-import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -50,7 +49,6 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
-import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
@@ -411,119 +409,4 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
         }
     }
     
-    @Test
-    public void testDroppingIndexedColDropsViewIndex() throws Exception {
-        helpTestDroppingIndexedColDropsViewIndex(false);
-    }
-    
-    @Test
-    public void testDroppingIndexedColDropsMultiTenantViewIndex() throws Exception {
-        helpTestDroppingIndexedColDropsViewIndex(true);
-    }
-    
-    public void helpTestDroppingIndexedColDropsViewIndex(boolean isMultiTenant) throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        props.setProperty(TENANT_ID_ATTRIB, TENANT_ID);
-        try (Connection conn = getConnection();
-                Connection viewConn = isMultiTenant ? getConnection(props) : conn ) {
-            String tableWithView = generateUniqueName();
-            String viewOfTable = generateUniqueName();
-            String viewIndex1 = generateUniqueName();
-            String viewIndex2 = generateUniqueName();
-            
-            conn.setAutoCommit(false);
-            viewConn.setAutoCommit(false);
-            String ddlFormat = "CREATE TABLE " + tableWithView + " (%s k VARCHAR NOT NULL, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR, v4 VARCHAR CONSTRAINT PK PRIMARY KEY(%s k))%s";
-            String ddl = String.format(ddlFormat, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
-                    isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");
-            conn.createStatement().execute(ddl);
-            viewConn.createStatement()
-                    .execute(
-                        "CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableWithView );
-            // create an index with the column that will be dropped
-            viewConn.createStatement().execute("CREATE INDEX " + viewIndex1 + " ON " + viewOfTable + "(v2) INCLUDE (v4)");
-            // create an index without the column that will be dropped
-            viewConn.createStatement().execute("CREATE INDEX " + viewIndex2 + " ON " + viewOfTable + "(v1) INCLUDE (v4)");
-            // verify index was created
-            try {
-                viewConn.createStatement().execute("SELECT * FROM " + viewIndex1 );
-            } catch (TableNotFoundException e) {
-                fail("Index on view was not created");
-            }
-            
-            // upsert a single row
-            PreparedStatement stmt = viewConn.prepareStatement("UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?,?,?)");
-            stmt.setString(1, "a");
-            stmt.setString(2, "b");
-            stmt.setString(3, "c");
-            stmt.setString(4, "d");
-            stmt.setString(5, "e");
-            stmt.setInt(6, 1);
-            stmt.setString(7, "g");
-            stmt.execute();
-            viewConn.commit();
-
-            // verify the index was created
-            PhoenixConnection pconn = viewConn.unwrap(PhoenixConnection.class);
-            PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null; 
-            PTable view = pconn.getTable(new PTableKey(tenantId,  viewOfTable ));
-            PTable viewIndex = pconn.getTable(new PTableKey(tenantId,  viewIndex1 ));
-            byte[] viewIndexPhysicalTable = viewIndex.getPhysicalName().getBytes();
-            assertNotNull("Can't find view index", viewIndex);
-            assertEquals("Unexpected number of indexes ", 2, view.getIndexes().size());
-            assertEquals("Unexpected index ",  viewIndex1 , view.getIndexes().get(0).getName()
-                    .getString());
-            assertEquals("Unexpected index ",  viewIndex2 , view.getIndexes().get(1).getName()
-                .getString());
-            
-            // drop two columns
-            conn.createStatement().execute("ALTER TABLE " + tableWithView + " DROP COLUMN v2, v3 ");
-            
-            // verify columns were dropped
-            try {
-                conn.createStatement().execute("SELECT v2 FROM " + tableWithView );
-                fail("Column should have been dropped");
-            } catch (ColumnNotFoundException e) {
-            }
-            try {
-                conn.createStatement().execute("SELECT v3 FROM " + tableWithView );
-                fail("Column should have been dropped");
-            } catch (ColumnNotFoundException e) {
-            }
-            
-            // verify index metadata was dropped
-            try {
-                viewConn.createStatement().execute("SELECT * FROM " + viewIndex1 );
-                fail("Index metadata should have been dropped");
-            } catch (TableNotFoundException e) {
-            }
-            
-            pconn = viewConn.unwrap(PhoenixConnection.class);
-            view = pconn.getTable(new PTableKey(tenantId,  viewOfTable ));
-            try {
-                viewIndex = pconn.getTable(new PTableKey(tenantId,  viewIndex1 ));
-                fail("View index should have been dropped");
-            } catch (TableNotFoundException e) {
-            }
-            assertEquals("Unexpected number of indexes ", 1, view.getIndexes().size());
-            assertEquals("Unexpected index ",  viewIndex2 , view.getIndexes().get(0).getName().getString());
-            
-            // verify that the physical index view table is *not* dropped
-            conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(viewIndexPhysicalTable);
-            
-            // scan the physical table and verify there is a single row for the second local index
-            Scan scan = new Scan();
-            HTable table = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(viewIndexPhysicalTable);
-            ResultScanner results = table.getScanner(scan);
-            Result result = results.next();
-            assertNotNull(result);
-            PTable viewIndexPTable = pconn.getTable(new PTableKey(pconn.getTenantId(), viewIndex2));
-            PColumn column = viewIndexPTable.getColumnForColumnName(IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V4"));
-            byte[] cq = column.getColumnQualifierBytes();
-            // there should be a single row belonging to VIEWINDEX2 
-            assertNotNull(viewIndex2 + " row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, cq));
-            assertNull(results.next());
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index 986c317..21fa312 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -422,7 +422,7 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         try {
-            String ddl = "create table test_table (char_pk varchar not null,"
+            String ddl = "create table " + generateUniqueName() + " (char_pk varchar not null,"
         		+ " int_col integer, long_col integer, int_col integer"
         		+ " constraint pk primary key (char_pk))";
             conn.createStatement().execute(ddl);
@@ -440,7 +440,7 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         try {
-            String ddl = "create table test_table (char_pk varchar not null,"
+            String ddl = "create table " + generateUniqueName() + " (char_pk varchar not null,"
         		+ " a.int_col integer, a.long_col integer,"
         		+ " a.int_col integer, b.long_col integer"
         		+ " constraint pk primary key (char_pk))";


[29/50] [abbrv] phoenix git commit: PHOENIX-4853 Add sql statement to PhoenixMetricsLog interface for query level metrics logging

Posted by ja...@apache.org.
PHOENIX-4853 Add sql statement to PhoenixMetricsLog interface for query level metrics logging


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

Branch: refs/heads/omid2
Commit: e979241ac9e7fb938c56a9e41b38f7d5aa3a56cd
Parents: eb79c5b
Author: Karan Mehta <ka...@gmail.com>
Authored: Fri Aug 17 13:02:08 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Fri Aug 17 17:36:28 2018 -0700

----------------------------------------------------------------------
 .../monitoring/PhoenixLoggingMetricsIT.java     | 15 +++++++++---
 .../phoenix/jdbc/LoggingPhoenixConnection.java  | 16 ++++++-------
 .../jdbc/LoggingPhoenixPreparedStatement.java   | 13 +++++++----
 .../phoenix/jdbc/LoggingPhoenixResultSet.java   | 10 ++++----
 .../phoenix/jdbc/LoggingPhoenixStatement.java   | 24 ++++++++++++++++----
 .../apache/phoenix/jdbc/PhoenixMetricsLog.java  |  6 ++---
 6 files changed, 56 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e979241a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
index 02640e7..97b2c5d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
@@ -43,6 +43,7 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     private String tableName1;
     private String tableName2;
     private LoggingPhoenixConnection loggedConn;
+    private String loggedSql;
 
     @Before
     public void beforeTest() throws Exception {
@@ -75,7 +76,10 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
         rs.close();
         assertTrue("Read metrics for not found for " + tableName1,
                 requestReadMetricsMap.get(tableName1).size() > 0);
+        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
+
         assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
+        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
 
         // run UPSERT SELECT to verify mutation metrics are logged
         String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
@@ -120,7 +124,10 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
         rs.close();
         assertTrue("Read metrics for not found for " + tableName1,
                 requestReadMetricsMap.get(tableName1).size() > 0);
+        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
+
         assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
+        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
 
         // run UPSERT SELECT to verify mutation metrics are logged
         String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
@@ -155,18 +162,20 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
         return new LoggingPhoenixConnection(conn, new PhoenixMetricsLog() {
             @Override
             public void logOverAllReadRequestMetrics(
-                    Map<MetricType, Long> overAllQueryMetrics) {
+                    Map<MetricType, Long> overAllQueryMetrics, String sql) {
                 overAllQueryMetricsMap.putAll(overAllQueryMetrics);
+                loggedSql = sql;
             }
 
             @Override
             public void logRequestReadMetrics(
-                    Map<String, Map<MetricType, Long>> requestReadMetrics) {
+                    Map<String, Map<MetricType, Long>> requestReadMetrics, String sql) {
                 requestReadMetricsMap.putAll(requestReadMetrics);
+                loggedSql = sql;
             }
 
             @Override
-            public void logWriteMetricsfoForMutations(
+            public void logWriteMetricsfoForMutationsSinceLastReset(
                     Map<String, Map<MetricType, Long>> mutationWriteMetrics) {
                 mutationWriteMetricsMap.putAll(mutationWriteMetrics);
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e979241a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
index 9a2e00f..37917e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
@@ -61,7 +61,7 @@ public class LoggingPhoenixConnection extends DelegateConnection {
     @Override
     public PreparedStatement prepareStatement(String sql) throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql),
-                phoenixMetricsLog);
+                phoenixMetricsLog, sql);
     }
 
     @Override
@@ -69,40 +69,40 @@ public class LoggingPhoenixConnection extends DelegateConnection {
             int resultSetConcurrency) throws SQLException {
         return new LoggingPhoenixPreparedStatement(
                 super.prepareStatement(sql, resultSetType, resultSetConcurrency),
-                phoenixMetricsLog);
+                phoenixMetricsLog, sql);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, int resultSetType,
             int resultSetConcurrency, int resultSetHoldability) throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, resultSetType,
-            resultSetConcurrency, resultSetHoldability), phoenixMetricsLog);
+            resultSetConcurrency, resultSetHoldability), phoenixMetricsLog, sql);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys)
             throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, autoGeneratedKeys),
-                phoenixMetricsLog);
+                phoenixMetricsLog, sql);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, columnIndexes),
-                phoenixMetricsLog);
+                phoenixMetricsLog, sql);
     }
 
     @Override
     public PreparedStatement prepareStatement(String sql, String[] columnNames)
             throws SQLException {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, columnNames),
-                phoenixMetricsLog);
+                phoenixMetricsLog, sql);
     }
 
     @Override
     public void commit() throws SQLException {
         super.commit();
-        phoenixMetricsLog.logWriteMetricsfoForMutations(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
+        phoenixMetricsLog.logWriteMetricsfoForMutationsSinceLastReset(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
         phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
         PhoenixRuntime.resetMetrics(conn);
     }
@@ -110,7 +110,7 @@ public class LoggingPhoenixConnection extends DelegateConnection {
     @Override
     public void close() throws SQLException {
         try {
-            phoenixMetricsLog.logWriteMetricsfoForMutations(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
+            phoenixMetricsLog.logWriteMetricsfoForMutationsSinceLastReset(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
             phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
             PhoenixRuntime.resetMetrics(conn);
         } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e979241a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
index cc8ec1f..952e3fd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixPreparedStatement.java
@@ -20,34 +20,37 @@ package org.apache.phoenix.jdbc;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
 
 public class LoggingPhoenixPreparedStatement extends DelegatePreparedStatement {
     
     private PhoenixMetricsLog phoenixMetricsLog;
+    private String sql;
     
-    public LoggingPhoenixPreparedStatement(PreparedStatement stmt, PhoenixMetricsLog phoenixMetricsLog) {
+    public LoggingPhoenixPreparedStatement(PreparedStatement stmt, PhoenixMetricsLog phoenixMetricsLog, String sql) {
         super(stmt);
         this.phoenixMetricsLog = phoenixMetricsLog;
+        this.sql = sql;
     }
     
     @Override
     public ResultSet executeQuery(String sql) throws SQLException {
-        return new LoggingPhoenixResultSet(super.executeQuery(sql), phoenixMetricsLog);
+        throw new SQLFeatureNotSupportedException();
     }
 
     @Override
     public ResultSet executeQuery() throws SQLException {
-        return new LoggingPhoenixResultSet(super.executeQuery(), phoenixMetricsLog);
+        return new LoggingPhoenixResultSet(super.executeQuery(), phoenixMetricsLog, sql);
     }
 
     @Override
     public ResultSet getResultSet() throws SQLException {
-        return new LoggingPhoenixResultSet(super.getResultSet(), phoenixMetricsLog);
+        return new LoggingPhoenixResultSet(super.getResultSet(), phoenixMetricsLog, sql);
     }
     
     @Override
     public ResultSet getGeneratedKeys() throws SQLException {
-        return new LoggingPhoenixResultSet(super.getGeneratedKeys(), phoenixMetricsLog);
+        return new LoggingPhoenixResultSet(super.getGeneratedKeys(), phoenixMetricsLog, sql);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e979241a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
index fbde499..53f5cb4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
@@ -25,16 +25,18 @@ import org.apache.phoenix.util.PhoenixRuntime;
 public class LoggingPhoenixResultSet extends DelegateResultSet {
     
     private PhoenixMetricsLog phoenixMetricsLog;
-    
-    public LoggingPhoenixResultSet(ResultSet rs, PhoenixMetricsLog phoenixMetricsLog) {
+    private String sql;
+
+    public LoggingPhoenixResultSet(ResultSet rs, PhoenixMetricsLog phoenixMetricsLog, String sql) {
         super(rs);
         this.phoenixMetricsLog = phoenixMetricsLog;
+        this.sql = sql;
     }
     
     @Override
     public void close() throws SQLException {
-        phoenixMetricsLog.logOverAllReadRequestMetrics(PhoenixRuntime.getOverAllReadRequestMetricInfo(rs));
-        phoenixMetricsLog.logRequestReadMetrics(PhoenixRuntime.getRequestReadMetricInfo(rs));
+        phoenixMetricsLog.logOverAllReadRequestMetrics(PhoenixRuntime.getOverAllReadRequestMetricInfo(rs), sql);
+        phoenixMetricsLog.logRequestReadMetrics(PhoenixRuntime.getRequestReadMetricInfo(rs), sql);
         PhoenixRuntime.resetMetrics(rs);
         super.close();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e979241a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
index ea919ed..f9abe6a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixStatement.java
@@ -24,6 +24,7 @@ import java.sql.Statement;
 public class LoggingPhoenixStatement extends DelegateStatement {
 
     private PhoenixMetricsLog phoenixMetricsLog;
+    private String sql;
     
     public LoggingPhoenixStatement(Statement stmt, PhoenixMetricsLog phoenixMetricsLog) {
         super(stmt);
@@ -31,18 +32,31 @@ public class LoggingPhoenixStatement extends DelegateStatement {
     }
 
     @Override
+    public boolean execute(String sql) throws SQLException {
+        this.sql = sql;
+        return super.execute(sql);
+    }
+
+    @Override
     public ResultSet executeQuery(String sql) throws SQLException {
-        return new LoggingPhoenixResultSet(super.executeQuery(sql), phoenixMetricsLog);
+        this.sql = sql;
+        return new LoggingPhoenixResultSet(super.executeQuery(sql), phoenixMetricsLog, this.sql);
     }
-    
+
+    @Override
+    public int executeUpdate(String sql) throws SQLException {
+        this.sql = sql;
+        return super.executeUpdate(sql);
+    }
+
     @Override
     public ResultSet getResultSet() throws SQLException {
-        return new LoggingPhoenixResultSet(super.getResultSet(), phoenixMetricsLog);
+        return new LoggingPhoenixResultSet(super.getResultSet(), phoenixMetricsLog, this.sql);
     }
     
     @Override
     public ResultSet getGeneratedKeys() throws SQLException {
-        return new LoggingPhoenixResultSet(super.getGeneratedKeys(), phoenixMetricsLog);
+        return new LoggingPhoenixResultSet(super.getGeneratedKeys(), phoenixMetricsLog, this.sql);
     }
-    
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e979241a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
index be08e52..96556ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
@@ -23,11 +23,11 @@ import org.apache.phoenix.monitoring.MetricType;
 
 public interface PhoenixMetricsLog {
 
-    void logOverAllReadRequestMetrics(Map<MetricType, Long> overAllQueryMetrics);
+    void logOverAllReadRequestMetrics(Map<MetricType, Long> overAllQueryMetrics, String sql);
 
-    void logRequestReadMetrics(Map<String, Map<MetricType, Long>> requestReadMetrics);
+    void logRequestReadMetrics(Map<String, Map<MetricType, Long>> requestReadMetrics, String sql);
 
-    void logWriteMetricsfoForMutations(Map<String, Map<MetricType, Long>> mutationWriteMetrics);
+    void logWriteMetricsfoForMutationsSinceLastReset(Map<String, Map<MetricType, Long>> mutationWriteMetrics);
 
     void logReadMetricInfoForMutationsSinceLastReset(Map<String, Map<MetricType, Long>> mutationReadMetrics);
 


[37/50] [abbrv] phoenix git commit: PHOENIX-3547 Supporting more number of indices per table.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 ab61826..938ae1f 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
@@ -3495,15 +3495,15 @@ public final class PTableProtos {
      */
     com.google.protobuf.ByteString getTenantId();
 
-    // optional int32 viewIndexId = 21;
+    // optional int64 viewIndexId = 21;
     /**
-     * <code>optional int32 viewIndexId = 21;</code>
+     * <code>optional int64 viewIndexId = 21;</code>
      */
     boolean hasViewIndexId();
     /**
-     * <code>optional int32 viewIndexId = 21;</code>
+     * <code>optional int64 viewIndexId = 21;</code>
      */
-    int getViewIndexId();
+    long getViewIndexId();
 
     // optional bytes indexType = 22;
     /**
@@ -3694,6 +3694,16 @@ public final class PTableProtos {
      * <code>optional int32 transactionProvider = 38;</code>
      */
     int getTransactionProvider();
+
+    // optional int32 viewIndexType = 39 [default = 5];
+    /**
+     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     */
+    boolean hasViewIndexType();
+    /**
+     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     */
+    int getViewIndexType();
   }
   /**
    * Protobuf type {@code PTable}
@@ -3859,7 +3869,7 @@ public final class PTableProtos {
             }
             case 168: {
               bitField0_ |= 0x00010000;
-              viewIndexId_ = input.readInt32();
+              viewIndexId_ = input.readInt64();
               break;
             }
             case 178: {
@@ -3950,6 +3960,11 @@ public final class PTableProtos {
               transactionProvider_ = input.readInt32();
               break;
             }
+            case 312: {
+              bitField1_ |= 0x00000002;
+              viewIndexType_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4395,19 +4410,19 @@ public final class PTableProtos {
       return tenantId_;
     }
 
-    // optional int32 viewIndexId = 21;
+    // optional int64 viewIndexId = 21;
     public static final int VIEWINDEXID_FIELD_NUMBER = 21;
-    private int viewIndexId_;
+    private long viewIndexId_;
     /**
-     * <code>optional int32 viewIndexId = 21;</code>
+     * <code>optional int64 viewIndexId = 21;</code>
      */
     public boolean hasViewIndexId() {
       return ((bitField0_ & 0x00010000) == 0x00010000);
     }
     /**
-     * <code>optional int32 viewIndexId = 21;</code>
+     * <code>optional int64 viewIndexId = 21;</code>
      */
-    public int getViewIndexId() {
+    public long getViewIndexId() {
       return viewIndexId_;
     }
 
@@ -4730,6 +4745,22 @@ public final class PTableProtos {
       return transactionProvider_;
     }
 
+    // optional int32 viewIndexType = 39 [default = 5];
+    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 39;
+    private int viewIndexType_;
+    /**
+     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     */
+    public boolean hasViewIndexType() {
+      return ((bitField1_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     */
+    public int getViewIndexType() {
+      return viewIndexType_;
+    }
+
     private void initFields() {
       schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       tableNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -4750,7 +4781,7 @@ public final class PTableProtos {
       viewStatement_ = com.google.protobuf.ByteString.EMPTY;
       physicalNames_ = java.util.Collections.emptyList();
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
-      viewIndexId_ = 0;
+      viewIndexId_ = 0L;
       indexType_ = com.google.protobuf.ByteString.EMPTY;
       statsTimeStamp_ = 0L;
       storeNulls_ = false;
@@ -4768,6 +4799,7 @@ public final class PTableProtos {
       encodedCQCounters_ = java.util.Collections.emptyList();
       useStatsForParallelization_ = false;
       transactionProvider_ = 0;
+      viewIndexType_ = 5;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4893,7 +4925,7 @@ public final class PTableProtos {
         output.writeBytes(20, tenantId_);
       }
       if (((bitField0_ & 0x00010000) == 0x00010000)) {
-        output.writeInt32(21, viewIndexId_);
+        output.writeInt64(21, viewIndexId_);
       }
       if (((bitField0_ & 0x00020000) == 0x00020000)) {
         output.writeBytes(22, indexType_);
@@ -4946,6 +4978,9 @@ public final class PTableProtos {
       if (((bitField1_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(38, transactionProvider_);
       }
+      if (((bitField1_ & 0x00000002) == 0x00000002)) {
+        output.writeInt32(39, viewIndexType_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -5038,7 +5073,7 @@ public final class PTableProtos {
       }
       if (((bitField0_ & 0x00010000) == 0x00010000)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(21, viewIndexId_);
+          .computeInt64Size(21, viewIndexId_);
       }
       if (((bitField0_ & 0x00020000) == 0x00020000)) {
         size += com.google.protobuf.CodedOutputStream
@@ -5108,6 +5143,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(38, transactionProvider_);
       }
+      if (((bitField1_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(39, viewIndexType_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -5304,6 +5343,11 @@ public final class PTableProtos {
         result = result && (getTransactionProvider()
             == other.getTransactionProvider());
       }
+      result = result && (hasViewIndexType() == other.hasViewIndexType());
+      if (hasViewIndexType()) {
+        result = result && (getViewIndexType()
+            == other.getViewIndexType());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -5395,7 +5439,7 @@ public final class PTableProtos {
       }
       if (hasViewIndexId()) {
         hash = (37 * hash) + VIEWINDEXID_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexId();
+        hash = (53 * hash) + hashLong(getViewIndexId());
       }
       if (hasIndexType()) {
         hash = (37 * hash) + INDEXTYPE_FIELD_NUMBER;
@@ -5465,6 +5509,10 @@ public final class PTableProtos {
         hash = (37 * hash) + TRANSACTIONPROVIDER_FIELD_NUMBER;
         hash = (53 * hash) + getTransactionProvider();
       }
+      if (hasViewIndexType()) {
+        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexType();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5623,7 +5671,7 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00020000);
         tenantId_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00040000);
-        viewIndexId_ = 0;
+        viewIndexId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00080000);
         indexType_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00100000);
@@ -5663,6 +5711,8 @@ public final class PTableProtos {
         bitField1_ = (bitField1_ & ~0x00000008);
         transactionProvider_ = 0;
         bitField1_ = (bitField1_ & ~0x00000010);
+        viewIndexType_ = 5;
+        bitField1_ = (bitField1_ & ~0x00000020);
         return this;
       }
 
@@ -5857,6 +5907,10 @@ public final class PTableProtos {
           to_bitField1_ |= 0x00000001;
         }
         result.transactionProvider_ = transactionProvider_;
+        if (((from_bitField1_ & 0x00000020) == 0x00000020)) {
+          to_bitField1_ |= 0x00000002;
+        }
+        result.viewIndexType_ = viewIndexType_;
         result.bitField0_ = to_bitField0_;
         result.bitField1_ = to_bitField1_;
         onBuilt();
@@ -6065,6 +6119,9 @@ public final class PTableProtos {
         if (other.hasTransactionProvider()) {
           setTransactionProvider(other.getTransactionProvider());
         }
+        if (other.hasViewIndexType()) {
+          setViewIndexType(other.getViewIndexType());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7323,35 +7380,35 @@ public final class PTableProtos {
         return this;
       }
 
-      // optional int32 viewIndexId = 21;
-      private int viewIndexId_ ;
+      // optional int64 viewIndexId = 21;
+      private long viewIndexId_ ;
       /**
-       * <code>optional int32 viewIndexId = 21;</code>
+       * <code>optional int64 viewIndexId = 21;</code>
        */
       public boolean hasViewIndexId() {
         return ((bitField0_ & 0x00080000) == 0x00080000);
       }
       /**
-       * <code>optional int32 viewIndexId = 21;</code>
+       * <code>optional int64 viewIndexId = 21;</code>
        */
-      public int getViewIndexId() {
+      public long getViewIndexId() {
         return viewIndexId_;
       }
       /**
-       * <code>optional int32 viewIndexId = 21;</code>
+       * <code>optional int64 viewIndexId = 21;</code>
        */
-      public Builder setViewIndexId(int value) {
+      public Builder setViewIndexId(long value) {
         bitField0_ |= 0x00080000;
         viewIndexId_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 viewIndexId = 21;</code>
+       * <code>optional int64 viewIndexId = 21;</code>
        */
       public Builder clearViewIndexId() {
         bitField0_ = (bitField0_ & ~0x00080000);
-        viewIndexId_ = 0;
+        viewIndexId_ = 0L;
         onChanged();
         return this;
       }
@@ -8177,6 +8234,39 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int32 viewIndexType = 39 [default = 5];
+      private int viewIndexType_ = 5;
+      /**
+       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       */
+      public boolean hasViewIndexType() {
+        return ((bitField1_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       */
+      public int getViewIndexType() {
+        return viewIndexType_;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       */
+      public Builder setViewIndexType(int value) {
+        bitField1_ |= 0x00000020;
+        viewIndexType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       */
+      public Builder clearViewIndexType() {
+        bitField1_ = (bitField1_ & ~0x00000020);
+        viewIndexType_ = 5;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PTable)
     }
 
@@ -8846,7 +8936,7 @@ public final class PTableProtos {
       "es\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n" +
       "\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030" +
       "\005 \001(\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\016" +
+      "\"\307\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016" +
       "tableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162" +
       "\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequ" +
       "enceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013p" +
@@ -8857,7 +8947,7 @@ public final class PTableProtos {
       "me\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTen" +
       "ant\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewState" +
       "ment\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010ten" +
-      "antId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tinde" +
+      "antId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\003\022\021\n\tinde" +
       "xType\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\ns" +
       "toreNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005" +
       "\022\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtran" +
@@ -8869,12 +8959,12 @@ public final class PTableProtos {
       "eme\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021enc" +
       "odedCQCounters\030$ \003(\0132\021.EncodedCQCounter\022" +
       "\"\n\032useStatsForParallelization\030% \001(\010\022\033\n\023t" +
-      "ransactionProvider\030& \001(\005\"6\n\020EncodedCQCou" +
-      "nter\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\004",
-      "VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apach" +
-      "e.phoenix.coprocessor.generatedB\014PTableP" +
-      "rotosH\001\210\001\001\240\001\001"
+      "ransactionProvider\030& \001(\005\022\030\n\rviewIndexTyp" +
+      "e\030\' \001(\005:\0015\"6\n\020EncodedCQCounter\022\021\n\tcolFam" +
+      "ily\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\004VIEW\020\002\022\t\n\005INDE" +
+      "X\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.copr" +
+      "ocessor.generatedB\014PTableProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8898,7 +8988,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", "TransactionProvider", });
+              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", "ViewIndexType", });
           internal_static_EncodedCQCounter_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_EncodedCQCounter_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
index f1b03f8..fdca334 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
@@ -2157,6 +2157,16 @@ public final class ServerCachingProtos {
      * <code>required int32 immutableStorageScheme = 21;</code>
      */
     int getImmutableStorageScheme();
+
+    // optional int32 viewIndexType = 22;
+    /**
+     * <code>optional int32 viewIndexType = 22;</code>
+     */
+    boolean hasViewIndexType();
+    /**
+     * <code>optional int32 viewIndexType = 22;</code>
+     */
+    int getViewIndexType();
   }
   /**
    * Protobuf type {@code IndexMaintainer}
@@ -2350,6 +2360,11 @@ public final class ServerCachingProtos {
               immutableStorageScheme_ = input.readInt32();
               break;
             }
+            case 176: {
+              bitField0_ |= 0x00010000;
+              viewIndexType_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2834,6 +2849,22 @@ public final class ServerCachingProtos {
       return immutableStorageScheme_;
     }
 
+    // optional int32 viewIndexType = 22;
+    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 22;
+    private int viewIndexType_;
+    /**
+     * <code>optional int32 viewIndexType = 22;</code>
+     */
+    public boolean hasViewIndexType() {
+      return ((bitField0_ & 0x00010000) == 0x00010000);
+    }
+    /**
+     * <code>optional int32 viewIndexType = 22;</code>
+     */
+    public int getViewIndexType() {
+      return viewIndexType_;
+    }
+
     private void initFields() {
       saltBuckets_ = 0;
       isMultiTenant_ = false;
@@ -2856,6 +2887,7 @@ public final class ServerCachingProtos {
       indexedColumnInfo_ = java.util.Collections.emptyList();
       encodingScheme_ = 0;
       immutableStorageScheme_ = 0;
+      viewIndexType_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3016,6 +3048,9 @@ public final class ServerCachingProtos {
       if (((bitField0_ & 0x00008000) == 0x00008000)) {
         output.writeInt32(21, immutableStorageScheme_);
       }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        output.writeInt32(22, viewIndexType_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -3114,6 +3149,10 @@ public final class ServerCachingProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(21, immutableStorageScheme_);
       }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(22, viewIndexType_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -3227,6 +3266,11 @@ public final class ServerCachingProtos {
         result = result && (getImmutableStorageScheme()
             == other.getImmutableStorageScheme());
       }
+      result = result && (hasViewIndexType() == other.hasViewIndexType());
+      if (hasViewIndexType()) {
+        result = result && (getViewIndexType()
+            == other.getViewIndexType());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -3324,6 +3368,10 @@ public final class ServerCachingProtos {
         hash = (37 * hash) + IMMUTABLESTORAGESCHEME_FIELD_NUMBER;
         hash = (53 * hash) + getImmutableStorageScheme();
       }
+      if (hasViewIndexType()) {
+        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexType();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3500,6 +3548,8 @@ public final class ServerCachingProtos {
         bitField0_ = (bitField0_ & ~0x00080000);
         immutableStorageScheme_ = 0;
         bitField0_ = (bitField0_ & ~0x00100000);
+        viewIndexType_ = 0;
+        bitField0_ = (bitField0_ & ~0x00200000);
         return this;
       }
 
@@ -3637,6 +3687,10 @@ public final class ServerCachingProtos {
           to_bitField0_ |= 0x00008000;
         }
         result.immutableStorageScheme_ = immutableStorageScheme_;
+        if (((from_bitField0_ & 0x00200000) == 0x00200000)) {
+          to_bitField0_ |= 0x00010000;
+        }
+        result.viewIndexType_ = viewIndexType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3815,6 +3869,9 @@ public final class ServerCachingProtos {
         if (other.hasImmutableStorageScheme()) {
           setImmutableStorageScheme(other.getImmutableStorageScheme());
         }
+        if (other.hasViewIndexType()) {
+          setViewIndexType(other.getViewIndexType());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5579,6 +5636,39 @@ public final class ServerCachingProtos {
         return this;
       }
 
+      // optional int32 viewIndexType = 22;
+      private int viewIndexType_ ;
+      /**
+       * <code>optional int32 viewIndexType = 22;</code>
+       */
+      public boolean hasViewIndexType() {
+        return ((bitField0_ & 0x00200000) == 0x00200000);
+      }
+      /**
+       * <code>optional int32 viewIndexType = 22;</code>
+       */
+      public int getViewIndexType() {
+        return viewIndexType_;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 22;</code>
+       */
+      public Builder setViewIndexType(int value) {
+        bitField0_ |= 0x00200000;
+        viewIndexType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 22;</code>
+       */
+      public Builder clearViewIndexType() {
+        bitField0_ = (bitField0_ & ~0x00200000);
+        viewIndexType_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:IndexMaintainer)
     }
 
@@ -8615,7 +8705,7 @@ public final class ServerCachingProtos {
       "ength\030\003 \002(\005\"4\n\017ColumnReference\022\016\n\006family" +
       "\030\001 \002(\014\022\021\n\tqualifier\030\002 \002(\014\"4\n\nColumnInfo\022" +
       "\022\n\nfamilyName\030\001 \001(\t\022\022\n\ncolumnName\030\002 \002(\t\"" +
-      "\306\005\n\017IndexMaintainer\022\023\n\013saltBuckets\030\001 \002(\005" +
+      "\335\005\n\017IndexMaintainer\022\023\n\013saltBuckets\030\001 \002(\005" +
       "\022\025\n\risMultiTenant\030\002 \002(\010\022\023\n\013viewIndexId\030\003" +
       " \001(\014\022(\n\016indexedColumns\030\004 \003(\0132\020.ColumnRef" +
       "erence\022 \n\030indexedColumnTypeOrdinal\030\005 \003(\005",
@@ -8632,23 +8722,23 @@ public final class ServerCachingProtos {
       "ed\030\020 \002(\010\022\033\n\023indexRowKeyByteSize\030\021 \002(\005\022\021\n" +
       "\timmutable\030\022 \002(\010\022&\n\021indexedColumnInfo\030\023 " +
       "\003(\0132\013.ColumnInfo\022\026\n\016encodingScheme\030\024 \002(\005" +
-      "\022\036\n\026immutableStorageScheme\030\025 \002(\005\"\334\001\n\025Add" +
-      "ServerCacheRequest\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007" +
-      "cacheId\030\002 \002(\014\022)\n\010cachePtr\030\003 \002(\0132\027.Immuta" +
-      "bleBytesWritable\022)\n\014cacheFactory\030\004 \002(\0132\023" +
-      ".ServerCacheFactory\022\017\n\007txState\030\005 \001(\014\022\"\n\032" +
-      "hasProtoBufIndexMaintainer\030\006 \001(\010\022\025\n\rclie" +
-      "ntVersion\030\007 \001(\005\"(\n\026AddServerCacheRespons",
-      "e\022\016\n\006return\030\001 \002(\010\"=\n\030RemoveServerCacheRe" +
-      "quest\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014" +
-      "\"+\n\031RemoveServerCacheResponse\022\016\n\006return\030" +
-      "\001 \002(\0102\245\001\n\024ServerCachingService\022A\n\016addSer" +
-      "verCache\022\026.AddServerCacheRequest\032\027.AddSe" +
-      "rverCacheResponse\022J\n\021removeServerCache\022\031" +
-      ".RemoveServerCacheRequest\032\032.RemoveServer" +
-      "CacheResponseBG\n(org.apache.phoenix.copr" +
-      "ocessor.generatedB\023ServerCachingProtosH\001" +
-      "\210\001\001\240\001\001"
+      "\022\036\n\026immutableStorageScheme\030\025 \002(\005\022\025\n\rview" +
+      "IndexType\030\026 \001(\005\"\334\001\n\025AddServerCacheReques" +
+      "t\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014\022)\n\010" +
+      "cachePtr\030\003 \002(\0132\027.ImmutableBytesWritable\022" +
+      ")\n\014cacheFactory\030\004 \002(\0132\023.ServerCacheFacto" +
+      "ry\022\017\n\007txState\030\005 \001(\014\022\"\n\032hasProtoBufIndexM" +
+      "aintainer\030\006 \001(\010\022\025\n\rclientVersion\030\007 \001(\005\"(",
+      "\n\026AddServerCacheResponse\022\016\n\006return\030\001 \002(\010" +
+      "\"=\n\030RemoveServerCacheRequest\022\020\n\010tenantId" +
+      "\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014\"+\n\031RemoveServerC" +
+      "acheResponse\022\016\n\006return\030\001 \002(\0102\245\001\n\024ServerC" +
+      "achingService\022A\n\016addServerCache\022\026.AddSer" +
+      "verCacheRequest\032\027.AddServerCacheResponse" +
+      "\022J\n\021removeServerCache\022\031.RemoveServerCach" +
+      "eRequest\032\032.RemoveServerCacheResponseBG\n(" +
+      "org.apache.phoenix.coprocessor.generated" +
+      "B\023ServerCachingProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8678,7 +8768,7 @@ public final class ServerCachingProtos {
           internal_static_IndexMaintainer_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_IndexMaintainer_descriptor,
-              new java.lang.String[] { "SaltBuckets", "IsMultiTenant", "ViewIndexId", "IndexedColumns", "IndexedColumnTypeOrdinal", "DataTableColRefForCoveredColumns", "IndexTableColRefForCoveredColumns", "IsLocalIndex", "IndexTableName", "RowKeyOrderOptimizable", "DataTableEmptyKeyValueColFamily", "EmptyKeyValueColFamily", "IndexedExpressions", "RowKeyMetadata", "NumDataTableColFamilies", "IndexWalDisabled", "IndexRowKeyByteSize", "Immutable", "IndexedColumnInfo", "EncodingScheme", "ImmutableStorageScheme", });
+              new java.lang.String[] { "SaltBuckets", "IsMultiTenant", "ViewIndexId", "IndexedColumns", "IndexedColumnTypeOrdinal", "DataTableColRefForCoveredColumns", "IndexTableColRefForCoveredColumns", "IsLocalIndex", "IndexTableName", "RowKeyOrderOptimizable", "DataTableEmptyKeyValueColFamily", "EmptyKeyValueColFamily", "IndexedExpressions", "RowKeyMetadata", "NumDataTableColFamilies", "IndexWalDisabled", "IndexRowKeyByteSize", "Immutable", "IndexedColumnInfo", "EncodingScheme", "ImmutableStorageScheme", "ViewIndexType", });
           internal_static_AddServerCacheRequest_descriptor =
             getDescriptor().getMessageTypes().get(4);
           internal_static_AddServerCacheRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 bc2523d..140c304 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,6 +101,7 @@ 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.schema.types.PLong;
 import org.apache.phoenix.util.BitSet;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
@@ -314,6 +315,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     }
     
     private byte[] viewIndexId;
+    private PDataType viewIndexType;
     private boolean isMultiTenant;
     // indexed expressions that are not present in the row key of the data table, the expression can also refer to a regular column
     private List<Expression> indexedExpressions;
@@ -371,7 +373,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         this(dataTable.getRowKeySchema(), dataTable.getBucketNum() != null);
         this.rowKeyOrderOptimizable = index.rowKeyOrderOptimizable();
         this.isMultiTenant = dataTable.isMultiTenant();
-        this.viewIndexId = index.getViewIndexId() == null ? null : MetaDataUtil.getViewIndexIdDataType().toBytes(index.getViewIndexId());
+        this.viewIndexId = index.getViewIndexId() == null ? null : index.getViewIndexType().toBytes(index.getViewIndexId());
+        this.viewIndexType = index.getViewIndexType();
         this.isLocalIndex = index.getIndexType() == IndexType.LOCAL;
         this.encodingScheme = index.getEncodingScheme();
         
@@ -823,7 +826,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
 
                 @Override
                 public PDataType getDataType() {
-                    return MetaDataUtil.getViewIndexIdDataType();
+                    return viewIndexType;
                 }
 
                 @Override
@@ -1220,7 +1223,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         boolean hasViewIndexId = encodedIndexedColumnsAndViewId < 0;
         if (hasViewIndexId) {
             // Fixed length
-            viewIndexId = new byte[MetaDataUtil.getViewIndexIdDataType().getByteSize()];
+            //Use legacy viewIndexIdType for clients older than 4.10 release
+            viewIndexId = new byte[MetaDataUtil.getLegacyViewIndexIdDataType().getByteSize()];
+            viewIndexType = MetaDataUtil.getLegacyViewIndexIdDataType();
             input.readFully(viewIndexId);
         }
         int nIndexedColumns = Math.abs(encodedIndexedColumnsAndViewId) - 1;
@@ -1337,6 +1342,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         maintainer.nIndexSaltBuckets = proto.getSaltBuckets();
         maintainer.isMultiTenant = proto.getIsMultiTenant();
         maintainer.viewIndexId = proto.hasViewIndexId() ? proto.getViewIndexId().toByteArray() : null;
+        maintainer.viewIndexType = proto.hasViewIndexType()
+                ? PDataType.fromTypeId(proto.getViewIndexType())
+                : MetaDataUtil.getLegacyViewIndexIdDataType();
         List<ServerCachingProtos.ColumnReference> indexedColumnsList = proto.getIndexedColumnsList();
         maintainer.indexedColumns = new HashSet<ColumnReference>(indexedColumnsList.size());
         for (ServerCachingProtos.ColumnReference colRefFromProto : indexedColumnsList) {
@@ -1456,6 +1464,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         builder.setIsMultiTenant(maintainer.isMultiTenant);
         if (maintainer.viewIndexId != null) {
             builder.setViewIndexId(ByteStringer.wrap(maintainer.viewIndexId));
+            builder.setViewIndexType(maintainer.viewIndexType.getSqlType());
         }
         for (ColumnReference colRef : maintainer.indexedColumns) {
             ServerCachingProtos.ColumnReference.Builder cRefBuilder =  ServerCachingProtos.ColumnReference.newBuilder();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index 397565b..4c2013b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -312,7 +312,7 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
                     new HashMap<ImmutableBytesWritable, String>();
             for (PTable index : indexes) {
                 if (localIndex == null) localIndex = index;
-                localIndexNames.put(new ImmutableBytesWritable(MetaDataUtil.getViewIndexIdDataType().toBytes(
+                localIndexNames.put(new ImmutableBytesWritable(index.getViewIndexType().toBytes(
                         index.getViewIndexId())), index.getName().getString());
             }
             if (localIndex == null) {
@@ -570,4 +570,4 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
index 1a22f60..2671044 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
@@ -47,6 +47,7 @@ import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.StringUtil;
 
@@ -204,15 +205,19 @@ public abstract class ExplainTable {
             range = ptr.get();
         }
         if (changeViewIndexId) {
-            Short s = (Short) type.toObject(range);
-            s = (short) (s + (-Short.MAX_VALUE));
-            buf.append(s.toString());
+            buf.append(getViewIndexValue(type, range).toString());
         } else {
             Format formatter = context.getConnection().getFormatter(type);
             buf.append(type.toStringLiteral(range, formatter));
         }
     }
-    
+
+    private Long getViewIndexValue(PDataType type, byte[] range) {
+        boolean useLongViewIndex = MetaDataUtil.getViewIndexIdDataType().equals(type);
+        Object s = type.toObject(range);
+        return (useLongViewIndex ? (Long) s : (Short) s) - (useLongViewIndex ? Long.MAX_VALUE : Short.MAX_VALUE);
+    }
+
     private static class RowKeyValueIterator implements Iterator<byte[]> {
         private final RowKeySchema schema;
         private ImmutableBytesWritable ptr = new ImmutableBytesWritable();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 7a4a481..1b5760d 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
@@ -220,6 +220,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final byte[] IS_VIEW_REFERENCED_BYTES = Bytes.toBytes(IS_VIEW_REFERENCED);
     public static final String VIEW_INDEX_ID = "VIEW_INDEX_ID";
     public static final byte[] VIEW_INDEX_ID_BYTES = Bytes.toBytes(VIEW_INDEX_ID);
+    public static final String VIEW_INDEX_ID_DATA_TYPE = "VIEW_INDEX_ID_DATA_TYPE";
+    public static final byte[] VIEW_INDEX_ID_DATA_TYPE_BYTES = Bytes.toBytes(VIEW_INDEX_ID_DATA_TYPE);
     public static final String BASE_COLUMN_COUNT = "BASE_COLUMN_COUNT";
     public static final byte[] BASE_COLUMN_COUNT_BYTES = Bytes.toBytes(BASE_COLUMN_COUNT);
     public static final String IS_ROW_TIMESTAMP = "IS_ROW_TIMESTAMP";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 8bbb379..736df6d 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
@@ -2994,6 +2994,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             addViewIndexToParentLinks(metaConnection);
             moveChildLinks(metaConnection);
         }
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
+            metaConnection = addColumnsIfNotExists(
+                    metaConnection,
+                    PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                    MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0,
+                    PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA_TYPE + " "
+                            + PInteger.INSTANCE.getSqlTypeName());
+        }
         return metaConnection;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 f088172..bbfacc0 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
@@ -297,7 +297,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         if (!allocateIndexId) {
             return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
         } else {
-            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null, Short.MIN_VALUE);
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null, Long.MIN_VALUE, MetaDataUtil.getViewIndexIdDataType());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 32fedc8..bbff343 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
@@ -191,7 +191,8 @@ public interface QueryConstants {
             DISABLE_WAL + " BOOLEAN,\n" +
             MULTI_TENANT + " BOOLEAN,\n" +
             VIEW_TYPE + " UNSIGNED_TINYINT,\n" +
-            VIEW_INDEX_ID + " SMALLINT,\n" +
+            VIEW_INDEX_ID + " BIGINT,\n" +
+            VIEW_INDEX_ID_DATA_TYPE + " INTEGER,\n" +
             // Column metadata (will be null for table row)
             DATA_TYPE + " INTEGER," +
             COLUMN_SIZE + " INTEGER," +
@@ -370,4 +371,4 @@ public interface QueryConstants {
 	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
 	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
     
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 d1b8f1e..50bb722 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.schema.types.PDataType;
 import org.apache.phoenix.transaction.TransactionFactory;
 
 public class DelegateTable implements PTable {
@@ -207,11 +208,16 @@ public class DelegateTable implements PTable {
     }
 
     @Override
-    public Short getViewIndexId() {
+    public Long getViewIndexId() {
         return delegate.getViewIndexId();
     }
 
     @Override
+    public PDataType getViewIndexType() {
+        return delegate.getViewIndexType();
+    }
+
+    @Override
     public PTableKey getKey() {
         return delegate.getKey();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 1114463..ecf3cd0 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
@@ -91,6 +91,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USE_STATS_FOR_PARA
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 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.VIEW_INDEX_ID_DATA_TYPE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
@@ -301,8 +302,9 @@ public class MetaDataClient {
                     GUIDE_POSTS_WIDTH + "," +
                     IMMUTABLE_STORAGE_SCHEME + "," +
                     ENCODING_SCHEME + "," +
-                    USE_STATS_FOR_PARALLELIZATION +
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    USE_STATS_FOR_PARALLELIZATION +"," +
+                    VIEW_INDEX_ID_DATA_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -1061,7 +1063,7 @@ public class MetaDataClient {
         }
     }
     
-    public MutationState createTable(CreateTableStatement statement, byte[][] splits, PTable parent, String viewStatement, ViewType viewType, byte[][] viewColumnConstants, BitSet isViewColumnReferenced) throws SQLException {
+    public MutationState createTable(CreateTableStatement statement, byte[][] splits, PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexType, byte[][] viewColumnConstants, BitSet isViewColumnReferenced) throws SQLException {
         TableName tableName = statement.getTableName();
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
@@ -1122,7 +1124,7 @@ public class MetaDataClient {
                         true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
-        table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, false, null, null, tableProps, commonFamilyProps);
+        table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewIndexType, viewColumnConstants, isViewColumnReferenced, false, null, null, tableProps, commonFamilyProps);
 
         if (table == null || table.getType() == PTableType.VIEW /*|| table.isTransactional()*/) {
             return new MutationState(0, 0, connection);
@@ -1680,7 +1682,7 @@ public class MetaDataClient {
                 PrimaryKeyConstraint pk = FACTORY.primaryKey(null, allPkColumns);
                 tableProps.put(MetaDataUtil.DATA_TABLE_NAME_PROP_NAME, dataTable.getName().getString());
                 CreateTableStatement tableStatement = FACTORY.createTable(indexTableName, statement.getProps(), columnDefs, pk, statement.getSplitNodes(), PTableType.INDEX, statement.ifNotExists(), null, null, statement.getBindCount(), null);
-                table = createTableInternal(tableStatement, splits, dataTable, null, null, null, null, allocateIndexId, statement.getIndexType(), asyncCreatedDate, tableProps, commonFamilyProps);
+                table = createTableInternal(tableStatement, splits, dataTable, null, null, MetaDataUtil.getViewIndexIdDataType(),null, null, allocateIndexId, statement.getIndexType(), asyncCreatedDate, tableProps, commonFamilyProps);
                 break;
             } catch (ConcurrentTableMutationException e) { // Can happen if parent data table changes while above is in progress
                 if (numRetries<5) {
@@ -1894,7 +1896,7 @@ public class MetaDataClient {
     }
 
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
-            final PTable parent, String viewStatement, ViewType viewType,
+            final PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, boolean allocateIndexId,
             IndexType indexType, Date asyncCreatedDate,
             Map<String,Object> tableProps,
@@ -2582,7 +2584,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, null, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, viewIndexType, 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);
             }
             
@@ -2747,6 +2749,7 @@ public class MetaDataClient {
             } else {
                 tableUpsert.setBoolean(28, useStatsForParallelizationProp);
             }
+            tableUpsert.setInt(29, Types.BIGINT);
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2860,7 +2863,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, transactionProvider, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelizationProp);
+                        viewIndexType, 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;
@@ -3938,7 +3941,7 @@ public class MetaDataClient {
                                 PTableImpl viewIndexTable = new PTableImpl(sharedTableState.getTenantId(),
                                         sharedTableState.getSchemaName(), sharedTableState.getTableName(), ts,
                                         table.getColumnFamilies(), sharedTableState.getColumns(),
-                                        sharedTableState.getPhysicalNames(), sharedTableState.getViewIndexId(),
+                                        sharedTableState.getPhysicalNames(), sharedTableState.getViewIndexType(), sharedTableState.getViewIndexId(),
                                         table.isMultiTenant(), table.isNamespaceMapped(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
                                 TableRef indexTableRef = new TableRef(viewIndexTable);
                                 PName indexTableTenantId = sharedTableState.getTenantId();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 bb81d76..1623175 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
@@ -700,7 +700,8 @@ public interface PTable extends PMetaDataEntity {
 
     ViewType getViewType();
     String getViewStatement();
-    Short getViewIndexId();
+    Long getViewIndexId();
+    PDataType getViewIndexType();
     PTableKey getKey();
 
     IndexType getIndexType();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 d575902..8d57945 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,10 +69,12 @@ import org.apache.phoenix.schema.types.PChar;
 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.PLong;
 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.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
@@ -138,7 +140,8 @@ public class PTableImpl implements PTable {
     private boolean storeNulls;
     private TransactionFactory.Provider transactionProvider;
     private ViewType viewType;
-    private Short viewIndexId;
+    private PDataType viewIndexType;
+    private Long viewIndexId;
     private int estimatedSize;
     private IndexType indexType;
     private int baseColumnCount;
@@ -214,7 +217,7 @@ public class PTableImpl implements PTable {
     
     // For indexes stored in shared physical tables
     public PTableImpl(PName tenantId, PName schemaName, PName tableName, long timestamp, List<PColumnFamily> families, 
-            List<PColumn> columns, List<PName> physicalNames, Short viewIndexId, boolean multiTenant, boolean isNamespaceMpped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
+            List<PColumn> columns, List<PName> physicalNames,PDataType viewIndexType, Long viewIndexId, boolean multiTenant, boolean isNamespaceMpped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme,
             EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         this.pkColumns = this.allColumns = Collections.emptyList();
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
@@ -228,7 +231,7 @@ public class PTableImpl implements PTable {
         this.families = families;
         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,
+            null, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
             transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
     }
 
@@ -293,7 +296,8 @@ public class PTableImpl implements PTable {
             view.getTenantId(), view.getSchemaName(), view.getTableName(), view.getType(), view.getIndexState(), timestamp,
             view.getSequenceNumber(), view.getPKName(), view.getBucketNum(), columns, view.getParentSchemaName(), view.getParentTableName(),
             view.getIndexes(), baseTable.isImmutableRows(), view.getPhysicalNames(), view.getDefaultFamilyName(), view.getViewStatement(),
-            baseTable.isWALDisabled(), baseTable.isMultiTenant(), baseTable.getStoreNulls(), view.getViewType(), view.getViewIndexId(), view.getIndexType(),
+            baseTable.isWALDisabled(), baseTable.isMultiTenant(), baseTable.getStoreNulls(), view.getViewType(),
+            view.getViewIndexType(), view.getViewIndexId(), view.getIndexType(),
             baseTableColumnCount, view.rowKeyOrderOptimizable(), baseTable.getTransactionProvider(), view.getUpdateCacheFrequency(),
             view.getIndexDisableTimestamp(), view.isNamespaceMapped(), baseTable.getAutoPartitionSeqName(), baseTable.isAppendOnlySchema(),
             baseTable.getImmutableStorageScheme(), baseTable.getEncodingScheme(), view.getEncodedCQCounter(), view.useStatsForParallelization());
@@ -340,7 +344,7 @@ public class PTableImpl implements PTable {
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
             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,
+            boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, PDataType viewIndexType, Long viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider,
             long updateCacheFrequency, long indexDisableTimestamp, boolean isNamespaceMapped,
             String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
@@ -348,7 +352,7 @@ public class PTableImpl implements PTable {
             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,
+                defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId,
                 indexType, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, transactionProvider,
                 updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName,
                 isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter,
@@ -359,7 +363,7 @@ public class PTableImpl implements PTable {
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
             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,
+            boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, PDataType viewIndexType, Long viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider,
             long updateCacheFrequency, int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
             String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
@@ -367,7 +371,7 @@ public class PTableImpl implements PTable {
             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,
+                defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType,viewIndexType,  viewIndexId,
                 indexType, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency,
                 indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme,
                 qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
@@ -380,7 +384,7 @@ public class PTableImpl implements PTable {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, table.getPKName(),
                 table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), indexes,
                 table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, viewStatement, table.isWALDisabled(),
-                table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
+                table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexType(), table.getViewIndexId(),
                 table.getIndexType(), baseTableColumnCount, rowKeyOrderOptimizable, table.getTransactionProvider(),
                 updateCacheFrequency, table.getIndexDisableTimestamp(), table.isNamespaceMapped(),
                 table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(),
@@ -391,13 +395,13 @@ public class PTableImpl implements PTable {
             long timeStamp, long sequenceNumber, 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,
+            boolean storeNulls, ViewType viewType, PDataType viewIndexType, Long viewIndexId, 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 {
         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,
+                viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
                 transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, 
                 qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
     }
@@ -431,7 +435,7 @@ public class PTableImpl implements PTable {
     private void init(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
             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,
+            boolean multiTenant, boolean storeNulls, ViewType viewType,PDataType viewIndexType,  Long viewIndexId,
             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 {
@@ -462,6 +466,7 @@ public class PTableImpl implements PTable {
         this.multiTenant = multiTenant;
         this.storeNulls = storeNulls;
         this.viewType = viewType;
+        this.viewIndexType = viewIndexType;
         this.viewIndexId = viewIndexId;
         this.indexType = indexType;
         this.transactionProvider = transactionProvider;
@@ -1203,11 +1208,16 @@ public class PTableImpl implements PTable {
     }
 
     @Override
-    public Short getViewIndexId() {
+    public Long getViewIndexId() {
         return viewIndexId;
     }
 
     @Override
+    public PDataType getViewIndexType() {
+        return viewIndexType;
+    }
+
+    @Override
     public PName getTenantId() {
         return tenantId;
     }
@@ -1235,10 +1245,13 @@ public class PTableImpl implements PTable {
         if (table.hasIndexState()) {
             indexState = PIndexState.fromSerializedValue(table.getIndexState());
         }
-        Short viewIndexId = null;
-        if(table.hasViewIndexId()){
-            viewIndexId = (short)table.getViewIndexId();
+        Long viewIndexId = null;
+        if (table.hasViewIndexId()) {
+            viewIndexId = table.getViewIndexId();
         }
+        PDataType viewIndexType = table.hasViewIndexType()
+                ? PDataType.fromTypeId(table.getViewIndexType())
+                : MetaDataUtil.getLegacyViewIndexIdDataType();
         IndexType indexType = IndexType.getDefault();
         if(table.hasIndexType()){
             indexType = IndexType.fromSerializedValue(table.getIndexType().toByteArray()[0]);
@@ -1352,7 +1365,7 @@ public class PTableImpl implements PTable {
             result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
                 (bucketNum == NO_SALTING) ? null : bucketNum, columns, parentSchemaName, parentTableName, indexes,
                         isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
-                        multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
+                        multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
                         transactionProvider, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, 
                         isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedColumnQualifierCounter, useStatsForParallelization);
             return result;
@@ -1375,7 +1388,8 @@ public class PTableImpl implements PTable {
         }
         if(table.getViewIndexId() != null) {
           builder.setViewIndexId(table.getViewIndexId());
-        }
+          builder.setViewIndexType(table.getViewIndexType().getSqlType());
+		}
         if(table.getIndexType() != null) {
             builder.setIndexType(ByteStringer.wrap(new byte[]{table.getIndexType().getSerializedValue()}));
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 f7b7b4a..b00ce77 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
@@ -576,6 +576,10 @@ public class MetaDataUtil {
     }
 
     public static PDataType getViewIndexIdDataType() {
+       return PLong.INSTANCE;
+    }
+
+    public static PDataType getLegacyViewIndexIdDataType() {
         return PSmallint.INSTANCE;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
index 3ab6a19..7b54cf1 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
@@ -50,7 +50,7 @@ public class TenantSpecificViewIndexCompileTest extends BaseConnectionlessQueryT
         conn.createStatement().execute("CREATE INDEX i1 ON v(v2) INCLUDE(v1)");
         
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT v1,v2 FROM v WHERE v2 > 'a' ORDER BY v2");
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32768,'me','a'] - [-32768,'me',*]",
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-9223372036854775808,'me','a'] - [-9223372036854775808,'me',*]",
                 QueryUtil.getExplainPlan(rs));
     }
 
@@ -194,7 +194,7 @@ public class TenantSpecificViewIndexCompileTest extends BaseConnectionlessQueryT
         conn.createStatement().execute("CREATE INDEX i1 ON v(v2)");
         
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM v WHERE v2 > 'a' and k2 = 'a' ORDER BY v2,k2");
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32768,'me','a'] - [-32768,'me',*]\n" + 
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-9223372036854775808,'me','a'] - [-9223372036854775808,'me',*]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY",
                 QueryUtil.getExplainPlan(rs));
         
@@ -227,7 +227,7 @@ public class TenantSpecificViewIndexCompileTest extends BaseConnectionlessQueryT
         
         // Confirm that a read-only view on an updatable view still optimizes out the read-only parts of the updatable view
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM v2 WHERE v3 > 'a' and k2 = 'a' ORDER BY v3,k2");
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32768,'me','a'] - [-32768,'me',*]",
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-9223372036854775808,'me','a'] - [-9223372036854775808,'me',*]",
                 QueryUtil.getExplainPlan(rs));
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 2e881b8..26caa0d 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
@@ -261,7 +261,7 @@ public class CorrelatePlanTest {
             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,
+                    false, Collections.<PName>emptyList(), null, null, false, false, false, null, null,
                     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();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 6bf298e..f400d0b 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
@@ -184,7 +184,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, null, 0, 0L, false, null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, EncodedCQCounter.NULL_COUNTER, true);
+                    false, null, 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/bef8d7df/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 13d8f1a..f10aa3c 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -60,7 +60,8 @@ message SharedTableState {
   required bytes tableName = 3;
   repeated PColumn columns = 4;  
   repeated bytes physicalNames = 5;
-  required int32 viewIndexId = 6;
+  required int64 viewIndexId = 6;
+  optional int32 viewIndexType = 7 [default = 5];
 }
 
 message MetaDataResponse {
@@ -75,7 +76,8 @@ message MetaDataResponse {
   repeated SharedTableState sharedTablesToDelete = 9;
   optional PSchema schema = 10;
   optional int64 autoPartitionNum = 11;
-  optional int32 viewIndexId = 12;
+  optional int64 viewIndexId = 12;
+  optional int32 viewIndexType = 13 [default = 5];
 }
 
 message GetTableRequest {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index bc868bc..be771a9 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -85,7 +85,7 @@ message PTable {
   optional bytes viewStatement = 18;
   repeated bytes physicalNames = 19;
   optional bytes tenantId = 20;
-  optional int32 viewIndexId = 21;
+  optional int64 viewIndexId = 21;
   optional bytes indexType = 22;
   optional int64 statsTimeStamp = 23;
   optional bool storeNulls = 24;
@@ -103,6 +103,7 @@ message PTable {
   repeated EncodedCQCounter encodedCQCounters = 36;
   optional bool useStatsForParallelization = 37;
   optional int32 transactionProvider = 38;
+  optional int32 viewIndexType = 39 [default = 5];
 }
 
 message EncodedCQCounter {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-protocol/src/main/ServerCachingService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/ServerCachingService.proto b/phoenix-protocol/src/main/ServerCachingService.proto
index c059a1a..d92f2cd 100644
--- a/phoenix-protocol/src/main/ServerCachingService.proto
+++ b/phoenix-protocol/src/main/ServerCachingService.proto
@@ -62,6 +62,7 @@ message IndexMaintainer {
   repeated ColumnInfo indexedColumnInfo = 19;
   required int32 encodingScheme = 20;
   required int32 immutableStorageScheme = 21;
+  optional int32 viewIndexType = 22 ;
 }
 
 message AddServerCacheRequest {


[46/50] [abbrv] phoenix git commit: PHOENIX-4839 IndexHalfStoreFileReaderGenerator throws NullPointerException (Aman Poonia)

Posted by ja...@apache.org.
PHOENIX-4839 IndexHalfStoreFileReaderGenerator throws NullPointerException (Aman Poonia)


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

Branch: refs/heads/omid2
Commit: 4d8c5d43c3a6d09e7338bc7eaf767af0dd012860
Parents: 3b03e1b
Author: Lars Hofhansl <la...@apache.org>
Authored: Fri Sep 14 12:38:37 2018 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Fri Sep 14 12:38:37 2018 -0700

----------------------------------------------------------------------
 .../regionserver/IndexHalfStoreFileReader.java  | 48 ++++++++++++++++----
 .../IndexHalfStoreFileReaderGenerator.java      | 12 ++---
 2 files changed, 43 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4d8c5d43/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
index 8bd0d72..273a1b0 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_START_ROW_SUFFIX;
+
 import java.io.IOException;
 import java.util.Map;
 
@@ -26,10 +28,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.index.IndexMaintainer;
 
 /**
@@ -56,8 +60,9 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
     private final Map<ImmutableBytesWritable, IndexMaintainer> indexMaintainers;
     private final byte[][] viewConstants;
     private final int offset;
-    private final HRegionInfo regionInfo;
+    private final HRegionInfo childRegionInfo;
     private final byte[] regionStartKeyInHFile;
+    private final HRegionInfo currentRegion;
 
     /**
      * @param fs
@@ -69,17 +74,19 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
      * @param conf
      * @param indexMaintainers
      * @param viewConstants
-     * @param regionInfo
+     * @param childRegionInfo
      * @param regionStartKeyInHFile
      * @param splitKey
+     * @param currentRegion
      * @throws IOException
      */
     public IndexHalfStoreFileReader(final FileSystem fs, final Path p, final CacheConfig cacheConf,
             final FSDataInputStreamWrapper in, long size, final Reference r,
             final Configuration conf,
             final Map<ImmutableBytesWritable, IndexMaintainer> indexMaintainers,
-            final byte[][] viewConstants, final HRegionInfo regionInfo,
-            byte[] regionStartKeyInHFile, byte[] splitKey) throws IOException {
+            final byte[][] viewConstants, final HRegionInfo childRegionInfo,
+            byte[] regionStartKeyInHFile, byte[] splitKey, HRegionInfo currentRegion)
+            throws IOException {
         super(fs, p, in, size, cacheConf, conf);
         this.splitkey = splitKey == null ? r.getSplitKey() : splitKey;
         // Is it top or bottom half?
@@ -87,9 +94,10 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
         this.splitRow = CellUtil.cloneRow(KeyValue.createKeyValueFromKey(splitkey));
         this.indexMaintainers = indexMaintainers;
         this.viewConstants = viewConstants;
-        this.regionInfo = regionInfo;
+        this.childRegionInfo = childRegionInfo;
         this.regionStartKeyInHFile = regionStartKeyInHFile;
         this.offset = regionStartKeyInHFile.length;
+        this.currentRegion = currentRegion;
     }
 
     public int getOffset() {
@@ -105,7 +113,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
     }
 
     public HRegionInfo getRegionInfo() {
-        return regionInfo;
+        return childRegionInfo;
     }
 
     public byte[] getRegionStartKeyInHFile() {
@@ -125,8 +133,30 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
     }
 
     @Override
-    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread, boolean isCompaction, long readPt) {
-        return new LocalIndexStoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), true,
-                getHFileReader().hasMVCCInfo(), readPt);
+    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread,
+                                                boolean isCompaction, long readPt) {
+        return new LocalIndexStoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction),
+                true, getHFileReader().hasMVCCInfo(), readPt);
+    }
+
+    @Override
+    public boolean passesKeyRangeFilter(Scan scan) {
+        if (scan.getAttribute(SCAN_START_ROW_SUFFIX) == null) {
+            // Scan from compaction.
+            return true;
+        }
+        byte[] startKey = currentRegion.getStartKey();
+        byte[] endKey = currentRegion.getEndKey();
+        // If the region start key is not the prefix of the scan start row then we can return empty
+        // scanners. This is possible during merge where one of the child region scan should not return any
+        // results as we go through merged region.
+        int prefixLength =
+                scan.getStartRow().length - scan.getAttribute(SCAN_START_ROW_SUFFIX).length;
+        if (Bytes.compareTo(scan.getStartRow(), 0, prefixLength,
+                (startKey.length == 0 ? new byte[endKey.length] : startKey), 0,
+                (startKey.length == 0 ? endKey.length : startKey.length)) != 0) {
+            return false;
+        }
+        return true;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4d8c5d43/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index 02ee0e0..e56dd9d 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -161,11 +161,11 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
                             indexMaintainer);
                     }
                 }
-                if(indexMaintainers.isEmpty()) return reader;
+                if (indexMaintainers.isEmpty()) return reader;
                 byte[][] viewConstants = getViewConstants(dataTable);
-                return new IndexHalfStoreFileReader(fs, p, cacheConf, in, size, r, ctx
-                        .getEnvironment().getConfiguration(), indexMaintainers, viewConstants,
-                        childRegion, regionStartKeyInHFile, splitKey);
+                return new IndexHalfStoreFileReader(fs, p, cacheConf, in, size, r,
+                        ctx.getEnvironment().getConfiguration(), indexMaintainers, viewConstants,
+                        childRegion, regionStartKeyInHFile, splitKey, region.getRegionInfo());
             } catch (ClassNotFoundException e) {
                 throw new IOException(e);
             } catch (SQLException e) {
@@ -246,10 +246,6 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
     /**
      * @param env
      * @param store Local Index store
-     * @param scan
-     * @param scanType
-     * @param earliestPutTs
-     * @param request
      * @return StoreScanner for new Local Index data for a passed store and Null if repair is not possible
      * @throws IOException
      */


[09/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java
new file mode 100644
index 0000000..51d3b86
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java
@@ -0,0 +1,80 @@
+/*
+ * 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 java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.curator.shaded.com.google.common.collect.Lists;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Base class for tests that run with split SYSTEM.CATALOG.
+ * 
+ */
+@Category(SplitSystemCatalogTests.class)
+public class SplitSystemCatalogIT extends BaseTest {
+
+    protected static String SCHEMA1 = "SCHEMA1";
+    protected static String SCHEMA2 = "SCHEMA2";
+    protected static String SCHEMA3 = "SCHEMA3";
+    protected static String SCHEMA4 = "SCHEMA4";
+
+    protected static String TENANT1 = "tenant1";
+    protected static String TENANT2 = "tenant2";
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        NUM_SLAVES_BASE = 6;
+        Map<String, String> props = Collections.emptyMap();
+        boolean splitSystemCatalog = (driver == null);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        // Split SYSTEM.CATALOG once after the mini-cluster is started
+        if (splitSystemCatalog) {
+            splitSystemCatalog();
+        }
+    }
+    
+    protected static void splitSystemCatalog() throws SQLException, Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        }
+        String tableName = "TABLE";
+        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1, tableName);
+        String fullTableName2 = SchemaUtil.getTableName(SCHEMA2, tableName);
+        String fullTableName3 = SchemaUtil.getTableName(SCHEMA3, tableName);
+        String fullTableName4 = SchemaUtil.getTableName(SCHEMA4, tableName);
+        ArrayList<String> tableList = Lists.newArrayList(fullTableName1, fullTableName2, fullTableName3);
+        Map<String, List<String>> tenantToTableMap = Maps.newHashMap();
+        tenantToTableMap.put(null, tableList);
+        tenantToTableMap.put(TENANT1, Lists.newArrayList(fullTableName2, fullTableName3));
+        tenantToTableMap.put(TENANT2, Lists.newArrayList(fullTableName4));
+        splitSystemCatalog(tenantToTableMap);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
new file mode 100644
index 0000000..27fc5c6
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
@@ -0,0 +1,11 @@
+package org.apache.phoenix.end2end;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface SplitSystemCatalogTests {
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
new file mode 100644
index 0000000..e25415a
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
@@ -0,0 +1,244 @@
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.analyzeTable;
+import static org.apache.phoenix.util.TestUtil.getAllSplits;
+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.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.ReadOnlyTableException;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+@RunWith(Parameterized.class)
+public class StatsEnabledSplitSystemCatalogIT extends BaseUniqueNamesOwnClusterIT {
+	
+	private String tableDDLOptions;
+	private boolean transactional;
+
+	public StatsEnabledSplitSystemCatalogIT(boolean transactional) {
+	        StringBuilder optionBuilder = new StringBuilder();
+	        this.transactional = transactional;
+	        if (transactional) {
+	            optionBuilder.append(" TRANSACTIONAL=true ");
+	        }
+	        this.tableDDLOptions = optionBuilder.toString();
+	    }
+
+	@Parameters(name = "transactional = {0}")
+	public static Collection<Boolean> data() {
+		return Arrays.asList(new Boolean[] { false, true });
+	}
+	
+	@BeforeClass
+    public static void doSetup() throws Exception {
+        NUM_SLAVES_BASE = 3;
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
+        props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+	/**
+     * Salted tests must be in their own test file to ensure that the underlying
+     * table is dropped. Otherwise, the splits may not be performed.
+     * TODO: we should throw in that case
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testSaltedUpdatableViewWithIndex() throws Exception {
+        testUpdatableViewWithIndex(3, false);
+    }
+
+    @Test
+    public void testSaltedUpdatableViewWithLocalIndex() throws Exception {
+        testUpdatableViewWithIndex(3, true);
+    }
+	
+	@Test
+    public void testNonSaltedUpdatableViewWithIndex() throws Exception {
+        testUpdatableViewWithIndex(null, false);
+    }
+    
+    @Test
+    public void testNonSaltedUpdatableViewWithLocalIndex() throws Exception {
+        testUpdatableViewWithIndex(null, true);
+    }
+    
+    @Test
+    public void testUpdatableOnUpdatableView() throws Exception {
+        String fullTableName = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+        String ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1 + " WHERE k3 = 2";
+        ViewIT.testUpdatableView(fullTableName, fullViewName1, fullViewName2, ddl, null, tableDDLOptions);
+        Connection conn = DriverManager.getConnection(getUrl());
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(109, rs.getInt(2));
+        assertEquals(2, rs.getInt(3));
+        assertFalse(rs.next());
+        
+        conn.createStatement().execute("UPSERT INTO " + fullViewName2 + "(k2) VALUES(122)");
+        conn.commit();
+        rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2 + " WHERE k2 >= 120");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(122, rs.getInt(2));
+        assertEquals(2, rs.getInt(3));
+        assertFalse(rs.next());
+        
+        try {
+            conn.createStatement().execute("UPSERT INTO " + fullViewName2 + "(k2,k3) VALUES(123,3)");
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN.getErrorCode(), e.getErrorCode());
+        }
+
+        try {
+            conn.createStatement().execute("UPSERT INTO " + fullViewName2 + "(k2,k3) select k2, 3 from " + fullViewName1);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN.getErrorCode(), e.getErrorCode());
+        }
+    }
+    
+    private void testUpdatableViewWithIndex(Integer saltBuckets, boolean localIndex) throws Exception {
+        String schemaName = TestUtil.DEFAULT_SCHEMA_NAME + "_" + generateUniqueName();
+        String tableName = "T_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String viewName = "V_" + generateUniqueName();
+        ViewIT.testUpdatableView(fullTableName, viewName, null, null, saltBuckets, tableDDLOptions);
+        Pair<String, Scan> pair = ViewIT.testUpdatableViewIndex(fullTableName, saltBuckets, localIndex, viewName);
+        Scan scan = pair.getSecond();
+        String physicalTableName = pair.getFirst();
+        // Confirm that dropping the view also deletes the rows in the index
+        if (saltBuckets == null) {
+            try (Connection conn = DriverManager.getConnection(getUrl())) {
+                HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices()
+                        .getTable(Bytes.toBytes(physicalTableName));
+                if (ScanUtil.isLocalIndex(scan)) {
+                    ScanUtil.setLocalIndexAttributes(scan, 0, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
+                            scan.getStartRow(), scan.getStopRow());
+                }
+                ResultScanner scanner = htable.getScanner(scan);
+                Result result = scanner.next();
+                // Confirm index has rows
+                assertTrue(result != null && !result.isEmpty());
+
+                conn.createStatement().execute("DROP VIEW " + viewName);
+
+                // Confirm index has no rows after view is dropped
+                scanner = htable.getScanner(scan);
+                result = scanner.next();
+                assertTrue(result == null || result.isEmpty());
+            }
+        }
+    }
+    
+    @Test
+    public void testReadOnlyOnReadOnlyView() throws Exception {
+        Connection earlierCon = DriverManager.getConnection(getUrl());
+        Connection conn = DriverManager.getConnection(getUrl());
+        String fullTableName = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+        String fullParentViewName = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+        
+        String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) "+ tableDDLOptions;
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW " + fullParentViewName + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullParentViewName + " WHERE k < 9";
+        conn.createStatement().execute(ddl);
+        
+        try {
+            conn.createStatement().execute("UPSERT INTO " + fullParentViewName + " VALUES(1)");
+            fail();
+        } catch (ReadOnlyTableException e) {
+            
+        }
+        for (int i = 0; i < 10; i++) {
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ")");
+        }
+        conn.commit();
+        
+        analyzeTable(conn, fullParentViewName, transactional);
+        
+        List<KeyRange> splits = getAllSplits(conn, fullParentViewName);
+        assertEquals(4, splits.size());
+        
+        int count = 0;
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
+        while (rs.next()) {
+            assertEquals(count++, rs.getInt(1));
+        }
+        assertEquals(10, count);
+        
+        count = 0;
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullParentViewName);
+        while (rs.next()) {
+            count++;
+            assertEquals(count + 5, rs.getInt(1));
+        }
+        assertEquals(4, count);
+        count = 0;
+        rs = earlierCon.createStatement().executeQuery("SELECT k FROM " + fullParentViewName);
+        while (rs.next()) {
+            count++;
+            assertEquals(count + 5, rs.getInt(1));
+        }
+        assertEquals(4, count);
+        
+        try {
+            conn.createStatement().execute("UPSERT INTO " + fullViewName + " VALUES(1)");
+            fail();
+        } catch (ReadOnlyTableException e) {
+            
+        } finally {
+            conn.close();
+        }
+
+        conn = DriverManager.getConnection(getUrl());
+        count = 0;
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullViewName);
+        while (rs.next()) {
+            count++;
+            assertEquals(count + 5, rs.getInt(1));
+        }
+        assertEquals(3, count);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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
index 689eb20..aa2d971 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,6 +17,22 @@
  */
 package org.apache.phoenix.end2end;
 
+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.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +44,11 @@ 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.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -36,14 +56,6 @@ 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;
@@ -60,11 +72,11 @@ public class SystemCatalogCreationOnConnectionIT {
 
     private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
       "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-      "SYSTEM.MUTEX", "SYSTEM.LOG"));
+      "SYSTEM.MUTEX", "SYSTEM.LOG", "SYSTEM.CHILD_LINK"));
 
     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"));
+        "SYSTEM:MUTEX", "SYSTEM:LOG", "SYSTEM:CHILD_LINK"));
 
     private static class PhoenixSysCatCreationServices extends ConnectionQueryServicesImpl {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
index 6f49518..8a41fad 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
@@ -23,6 +23,8 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Collections;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -30,30 +32,34 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.junit.After;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.common.collect.Maps;
+
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogIT extends BaseTest {
     private HBaseTestingUtility testUtil = null;
-
-    @After
-    public void cleanup() throws Exception {
-        if (null != testUtil) {
-          testUtil.shutdownMiniCluster();
-          testUtil = null;
-        }
-    }
+    
+	@BeforeClass
+	public static void doSetup() throws Exception {
+		Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+		serverProps.put(QueryServices.SYSTEM_CATALOG_SPLITTABLE, "false");
+		Map<String, String> clientProps = Collections.emptyMap();
+		setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
+				new ReadOnlyProps(clientProps.entrySet().iterator()));
+	}
 
     /**
-     * Make sure that SYSTEM.CATALOG cannot be split, even with schemas and multi-tenant views
+     * Make sure that SYSTEM.CATALOG cannot be split if QueryServices.SYSTEM_CATALOG_SPLITTABLE is false
      */
     @Test
     public void testSystemTableSplit() throws Exception {
-        testUtil = new HBaseTestingUtility();
-        testUtil.startMiniCluster(1);
+        testUtil = getUtility();
         for (int i=0; i<10; i++) {
             createTable("schema"+i+".table_"+i);
         }
@@ -63,7 +69,6 @@ public class SystemCatalogIT extends BaseTest {
         try {
             // now attempt to split SYSTEM.CATALOG
             testUtil.getHBaseAdmin().split(systemCatalog);
-
             // make sure the split finishes (there's no synchronous splitting before HBase 2.x)
             testUtil.getHBaseAdmin().disableTable(systemCatalog);
             testUtil.getHBaseAdmin().enableTable(systemCatalog);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 34a1312..dd6f7f7 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
@@ -379,11 +379,12 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
 			connTenant2 = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2, props);
 	        validateTenantViewIsDropped(connTenant2);
 	        
+	        // TODO uncomment after PHOENIX-4764 is implemented
 	        // Validate Tenant Metadata is gone for the Tenant Table TENANT_TABLE_NAME
-            rs = meta.getTables(null, "", StringUtil.escapeLike(TENANT_TABLE_NAME), new String[] {PTableType.VIEW.getValue().getString()});
-            assertFalse(rs.next());
-            rs = meta.getTables(null, "", StringUtil.escapeLike(tenantTable2), new String[] {PTableType.VIEW.getValue().getString()});
-            assertFalse(rs.next());
+//            rs = meta.getTables(null, "", StringUtil.escapeLike(TENANT_TABLE_NAME), new String[] {PTableType.VIEW.getValue().getString()});
+//            assertFalse(rs.next());
+//            rs = meta.getTables(null, "", StringUtil.escapeLike(tenantTable2), new String[] {PTableType.VIEW.getValue().getString()});
+//            assertFalse(rs.next());
             
             rs = meta.getTables(null, "", StringUtil.escapeLike(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID), new String[] {PTableType.VIEW.getValue().getString()});
             assertTrue(rs.next());
@@ -491,6 +492,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, PTableType.SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE, PTableType.SYSTEM);
+            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);
@@ -530,7 +533,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             // (tenant_id column is not visible in tenant-specific connection)
             assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "tenant_type_id", 2);
-            assertEquals(1, rs.getInt(KEY_SEQ));
+            assertEquals(1, rs.getShort(KEY_SEQ));
             assertTrue(rs.next());
             assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "id", 3);
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index d5da0aa..31f3569 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -29,20 +29,23 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PNameFactory;
-import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 
 public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
 	
@@ -115,38 +118,40 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
     }
 
     private void testMultiCFViewIndex(boolean localIndex, boolean isNamespaceEnabled) throws Exception {
-        String tableName = "XYZ." + generateUniqueName();
-        String baseViewName = generateUniqueName() ;
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String viewName2 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
         createTableAndValidate(tableName, isNamespaceEnabled);
-        createViewAndIndexesWithTenantId(tableName, baseViewName, localIndex, "b", isNamespaceEnabled);
-        createViewAndIndexesWithTenantId(tableName, baseViewName, localIndex, "a", isNamespaceEnabled);
-
-        String sequenceNameA = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("a"), isNamespaceEnabled);
-        String sequenceNameB = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("b"), isNamespaceEnabled);
+        String tenantId1 = TENANT1;
+        String tenantId2 = TENANT2;
+        createViewAndIndexesWithTenantId(tableName, viewName1, localIndex, tenantId1, isNamespaceEnabled);
+        createViewAndIndexesWithTenantId(tableName, viewName2, localIndex, tenantId2, isNamespaceEnabled);
+        
+        String sequenceNameA = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName(tenantId2), isNamespaceEnabled);
+        String sequenceNameB = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName(tenantId1), isNamespaceEnabled);
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceEnabled);
-        verifySequenceValue(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName, -32767);
-        verifySequenceValue(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName, -32767);
+        verifySequenceValue(isNamespaceEnabled? tenantId2 : null, sequenceNameA, sequenceSchemaName, -32767);
+        verifySequenceValue(isNamespaceEnabled? tenantId1 : null, sequenceNameB, sequenceSchemaName, -32767);
 
         Properties props = new Properties();
-        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "a");
+        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId2);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.createStatement().execute("DROP VIEW  " + baseViewName + "_a");
+            conn.createStatement().execute("DROP VIEW  " + viewName2);
         }
-        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "b");
+        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId1);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.createStatement().execute("DROP VIEW  " + baseViewName + "_b");
+            conn.createStatement().execute("DROP VIEW  " + viewName1);
         }
         DriverManager.getConnection(getUrl()).createStatement().execute("DROP TABLE " + tableName + " CASCADE");
 
-        verifySequenceNotExists(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName);
-        verifySequenceNotExists(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName);
+        verifySequenceNotExists(isNamespaceEnabled? tenantId2 : null, sequenceNameA, sequenceSchemaName);
+        verifySequenceNotExists(isNamespaceEnabled? tenantId1 : null, sequenceNameB, sequenceSchemaName);
     }
 
-    private void createViewAndIndexesWithTenantId(String tableName,String baseViewName, boolean localIndex, String tenantId,
+    private void createViewAndIndexesWithTenantId(String tableName, String viewName, boolean localIndex, String tenantId,
             boolean isNamespaceMapped) throws Exception {
         Properties props = new Properties();
-        String viewName = baseViewName + "_" + tenantId;
-        String indexName = "idx_" + viewName;
+        String indexName = "I_"+ generateUniqueName();
         if (tenantId != null) {
             props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
         }
@@ -239,15 +244,15 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         assertFalse(rs.next());
 
         conn.close();
-
     }
     
     @Test
     public void testNonPaddedTenantId() throws Exception {
-        String tenantId1 = "org1";
-        String tenantId2 = "org2";
-        String tableName = generateUniqueName();
-        String viewName = generateUniqueName();
+        String tenantId1 = TENANT1;
+        String tenantId2 = TENANT2;
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        
         String ddl = "CREATE TABLE " + tableName + " (tenantId char(15) NOT NULL, pk1 varchar NOT NULL, pk2 INTEGER NOT NULL, val1 VARCHAR CONSTRAINT pk primary key (tenantId,pk1,pk2)) MULTI_TENANT = true";
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute(ddl);
@@ -288,10 +293,11 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
     }
     
     @Test
-    public void testOverlappingDatesFilter() throws SQLException {
-        String tenantUrl = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1" + ";" + QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB + "=true";
-        String tableName = generateUniqueName();
-        String viewName = generateUniqueName();
+    public void testOverlappingDatesFilter() throws Exception {
+        String tenantId = TENANT1;
+        String tenantUrl = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + tenantId + ";" + QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB + "=true";
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         String ddl = "CREATE TABLE " + tableName 
                 + "(ORGANIZATION_ID CHAR(15) NOT NULL, "
                 + "PARENT_TYPE CHAR(3) NOT NULL, "
@@ -315,7 +321,9 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
                     + "ORDER BY PARENT_TYPE,CREATED_DATE LIMIT 501";
             
             ResultSet rs = viewConn.createStatement().executeQuery(query);
-            String expectedPlanFormat = "CLIENT SERIAL 1-WAY RANGE SCAN OVER IDX ['tenant1        ','001','%s 00:00:00.001'] - ['tenant1        ','001','%s 00:00:00.000']" + "\n" +
+            String exptectedIndexName = SchemaUtil.getTableName(SCHEMA1, "IDX");
+            String expectedPlanFormat = "CLIENT SERIAL 1-WAY RANGE SCAN OVER " + exptectedIndexName
+                    + " ['tenant1        ','001','%s 00:00:00.001'] - ['tenant1        ','001','%s 00:00:00.000']" + "\n" +
                         "    SERVER FILTER BY FIRST KEY ONLY" + "\n" +
                         "    SERVER 501 ROW LIMIT" + "\n" +
                         "CLIENT 501 ROW LIMIT";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 48a49b2..2b866a5 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
@@ -17,13 +17,9 @@
  */
 package org.apache.phoenix.end2end;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.phoenix.query.ConnectionQueryServicesImpl.UPGRADE_MUTEX;
 import static org.apache.phoenix.query.ConnectionQueryServicesImpl.UPGRADE_MUTEX_UNLOCKED;
-import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.util.UpgradeUtil.SELECT_BASE_COLUMN_COUNT_FROM_HEADER_ROW;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -36,8 +32,6 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -47,11 +41,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.curator.shaded.com.google.common.collect.Sets;
-import org.apache.hadoop.hbase.HConstants;
 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.client.RowMutations;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -77,91 +69,10 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.apache.phoenix.util.UpgradeUtil;
-import org.junit.Before;
 import org.junit.Test;
 
 public class UpgradeIT extends ParallelStatsDisabledIT {
 
-    private String tenantId;
-    
-    @Before
-    public void generateTenantId() {
-        tenantId = "T_" + generateUniqueName();
-    }
-
-    @Test
-    public void testUpgradeForTenantViewWithSameColumnsAsBaseTable() throws Exception {
-        String tableWithViewName = generateUniqueName();
-        String viewTableName = generateUniqueName();
-        testViewUpgrade(true, tenantId, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.EQUAL);
-        testViewUpgrade(true, tenantId, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
-            ColumnDiff.EQUAL);
-        testViewUpgrade(true, tenantId, null, tableWithViewName + "3", viewTableName + "SCHEMA", viewTableName + "3",
-            ColumnDiff.EQUAL);
-        testViewUpgrade(true, tenantId, "TABLESCHEMA", tableWithViewName + "4", viewTableName + "SCHEMA", viewTableName + "4",
-            ColumnDiff.EQUAL);
-        testViewUpgrade(true, tenantId, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
-            ColumnDiff.EQUAL);
-    }
-
-    @Test
-    public void testUpgradeForTenantViewWithMoreColumnsThanBaseTable() throws Exception {
-        String tableWithViewName = generateUniqueName();
-        String viewTableName = generateUniqueName();
-        testViewUpgrade(true, tenantId, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.MORE);
-        testViewUpgrade(true, tenantId, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
-            ColumnDiff.MORE);
-        testViewUpgrade(true, tenantId, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3",
-            ColumnDiff.MORE);
-        testViewUpgrade(true, tenantId, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
-            ColumnDiff.MORE);
-        testViewUpgrade(true, tenantId, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
-            ColumnDiff.MORE);
-    }
-
-    @Test
-    public void testUpgradeForViewWithSameColumnsAsBaseTable() throws Exception {
-        String tableWithViewName = generateUniqueName();
-        String viewTableName = generateUniqueName();
-        testViewUpgrade(false, null, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
-            ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3",
-            ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
-            ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
-            ColumnDiff.EQUAL);
-    }
-
-    @Test
-    public void testUpgradeForViewWithMoreColumnsThanBaseTable() throws Exception {
-        String tableWithViewName = generateUniqueName();
-        String viewTableName = generateUniqueName();
-        testViewUpgrade(false, null, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.MORE);
-        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2", ColumnDiff.MORE);
-        testViewUpgrade(false, null, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3", ColumnDiff.MORE);
-        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
-            ColumnDiff.MORE);
-        testViewUpgrade(false, null, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
-            ColumnDiff.MORE);
-    }
-
-    @Test
-    public void testSettingBaseColumnCountWhenBaseTableColumnDropped() throws Exception {
-        String tableWithViewName = generateUniqueName();
-        String viewTableName = generateUniqueName();
-        testViewUpgrade(true, tenantId, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.MORE);
-        testViewUpgrade(true, tenantId, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
-            ColumnDiff.LESS);
-        testViewUpgrade(true, tenantId, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3",
-            ColumnDiff.LESS);
-        testViewUpgrade(true, tenantId, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
-            ColumnDiff.LESS);
-        testViewUpgrade(true, tenantId, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
-            ColumnDiff.LESS);
-    }
-
     @Test
     public void testMapTableToNamespaceDuringUpgrade()
             throws SQLException, IOException, IllegalArgumentException, InterruptedException {
@@ -232,7 +143,6 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
             admin.close();
             PhoenixConnection phxConn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
             UpgradeUtil.upgradeTable(phxConn, phoenixFullTableName);
-            UpgradeUtil.mapChildViewsToNamespace(phxConn, phoenixFullTableName,props);
             phxConn.close();
             props = new Properties();
             phxConn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
@@ -366,7 +276,6 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         props.setProperty(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, Boolean.toString(false));
         PhoenixConnection phxConn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
         UpgradeUtil.upgradeTable(phxConn, phoenixFullTableName);
-        UpgradeUtil.mapChildViewsToNamespace(phxConn,phoenixFullTableName,props);
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
         phxConn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
         // purge MetaDataCache except for system tables
@@ -423,222 +332,7 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         assertTrue(rs.next());
         assertTrue(rs.getString(1).contains(hbaseTableName));
     }
-    
-
-    @Test
-    public void testSettingBaseColumnCountForMultipleViewsOnTable() throws Exception {
-        String baseSchema = "S_" + generateUniqueName();
-        String baseTable = "T_" + generateUniqueName();
-        String fullBaseTableName = SchemaUtil.getTableName(baseSchema, baseTable);
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE " + fullBaseTableName + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 INTEGER, V2 INTEGER CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true";
-            conn.createStatement().execute(baseTableDDL);
-
-            String[] tenants = new String[] {"T_" + generateUniqueName(), "T_" + generateUniqueName()};
-            Collections.sort(Arrays.asList(tenants));
-            String[] tenantViews = new String[] {"V_" + generateUniqueName(), "V_" + generateUniqueName(), "V_" + generateUniqueName()};
-            Collections.sort(Arrays.asList(tenantViews));
-            String[] globalViews = new String[] {"G_" + generateUniqueName(), "G_" + generateUniqueName(), "G_" + generateUniqueName()};
-            Collections.sort(Arrays.asList(globalViews));
-            for (int i = 0; i < 2; i++) {
-                // Create views for tenants;
-                String tenant = tenants[i];
-                try (Connection tenantConn = createTenantConnection(tenant)) {
-                    String view = tenantViews[0];
-                    // view with its own column
-                    String viewDDL = "CREATE VIEW " + view + " AS SELECT * FROM " + fullBaseTableName;
-                    tenantConn.createStatement().execute(viewDDL);
-                    String addCols = "ALTER VIEW " + view + " ADD COL1 VARCHAR ";
-                    tenantConn.createStatement().execute(addCols);
-                    removeBaseColumnCountKV(tenant, null, view);
-
-                    // view that has the last base table column removed
-                    view = tenantViews[1];
-                    viewDDL = "CREATE VIEW " + view + " AS SELECT * FROM " + fullBaseTableName;
-                    tenantConn.createStatement().execute(viewDDL);
-                    String droplastBaseCol = "ALTER VIEW " + view + " DROP COLUMN V2";
-                    tenantConn.createStatement().execute(droplastBaseCol);
-                    removeBaseColumnCountKV(tenant, null, view);
-
-                    // view that has the middle base table column removed
-                    view = tenantViews[2];
-                    viewDDL = "CREATE VIEW " + view + " AS SELECT * FROM " + fullBaseTableName;
-                    tenantConn.createStatement().execute(viewDDL);
-                    String dropMiddileBaseCol = "ALTER VIEW " + view + " DROP COLUMN V1";
-                    tenantConn.createStatement().execute(dropMiddileBaseCol);
-                    removeBaseColumnCountKV(tenant, null, view);
-                }
-            }
-
-            // create global views
-            try (Connection globalConn = DriverManager.getConnection(getUrl())) {
-                String globalView = globalViews[0];
-                // view with its own column
-                String viewDDL = "CREATE VIEW " + globalView + " AS SELECT * FROM " + fullBaseTableName;
-                globalConn.createStatement().execute(viewDDL);
-                String addCols = "ALTER VIEW " + globalView + " ADD COL1 VARCHAR ";
-                globalConn.createStatement().execute(addCols);
-                removeBaseColumnCountKV(null, null, globalView);
-
-                // view that has the last base table column removed
-                globalView = globalViews[1];
-                viewDDL = "CREATE VIEW " + globalView + " AS SELECT * FROM " + fullBaseTableName;
-                globalConn.createStatement().execute(viewDDL);
-                String droplastBaseCol = "ALTER VIEW " + globalView + " DROP COLUMN V2";
-                globalConn.createStatement().execute(droplastBaseCol);
-                removeBaseColumnCountKV(null, null, globalView);
-
-                // view that has the middle base table column removed
-                globalView = globalViews[2];
-                viewDDL = "CREATE VIEW " + globalView + " AS SELECT * FROM " + fullBaseTableName;
-                globalConn.createStatement().execute(viewDDL);
-                String dropMiddileBaseCol = "ALTER VIEW " + globalView + " DROP COLUMN V1";
-                globalConn.createStatement().execute(dropMiddileBaseCol);
-                removeBaseColumnCountKV(null, null, globalView);
-            }
-            
-            // run upgrade
-            upgradeTo4_5_0(conn);
-            
-            // Verify base column counts for tenant specific views
-            for (int i = 0; i < 2 ; i++) {
-                String tenantId = tenants[i];
-                checkBaseColumnCount(tenantId, null, tenantViews[0], 4);
-                checkBaseColumnCount(tenantId, null, tenantViews[1], DIVERGED_VIEW_BASE_COLUMN_COUNT);
-                checkBaseColumnCount(tenantId, null, tenantViews[2], DIVERGED_VIEW_BASE_COLUMN_COUNT);
-            }
-            
-            // Verify base column count for global views
-            checkBaseColumnCount(null, null, globalViews[0], 4);
-            checkBaseColumnCount(null, null, globalViews[1], DIVERGED_VIEW_BASE_COLUMN_COUNT);
-            checkBaseColumnCount(null, null, globalViews[2], DIVERGED_VIEW_BASE_COLUMN_COUNT);
-        }
-        
         
-    }
-
-    private static void upgradeTo4_5_0(Connection conn) throws SQLException {
-        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        pconn.setRunningUpgrade(true);
-        UpgradeUtil.upgradeTo4_5_0(pconn);
-    }
-    
-    private enum ColumnDiff {
-        MORE, EQUAL, LESS
-    };
-
-    private void testViewUpgrade(boolean tenantView, String tenantId, String baseTableSchema,
-            String baseTableName, String viewSchema, String viewName, ColumnDiff diff)
-            throws Exception {
-        if (tenantView) {
-            checkNotNull(tenantId);
-        } else {
-            checkArgument(tenantId == null);
-        }
-        Connection conn = DriverManager.getConnection(getUrl());
-        String fullViewName = SchemaUtil.getTableName(viewSchema, viewName);
-        String fullBaseTableName = SchemaUtil.getTableName(baseTableSchema, baseTableName);
-        try {
-            int expectedBaseColumnCount;
-            conn.createStatement().execute(
-                "CREATE TABLE IF NOT EXISTS " + fullBaseTableName + " ("
-                        + " TENANT_ID CHAR(15) NOT NULL, " + " PK1 integer NOT NULL, "
-                        + "PK2 bigint NOT NULL, " + "CF1.V1 VARCHAR, " + "CF2.V2 VARCHAR, "
-                        + "V3 CHAR(100) ARRAY[4] "
-                        + " CONSTRAINT NAME_PK PRIMARY KEY (TENANT_ID, PK1, PK2)"
-                        + " ) MULTI_TENANT= true");
-            
-            // create a view with same columns as base table.
-            try (Connection conn2 = getConnection(tenantView, tenantId)) {
-                conn2.createStatement().execute(
-                    "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullBaseTableName);
-            }
-
-            if (diff == ColumnDiff.MORE) {
-                    // add a column to the view
-                    try (Connection conn3 = getConnection(tenantView, tenantId)) {
-                        conn3.createStatement().execute(
-                            "ALTER VIEW " + fullViewName + " ADD VIEW_COL1 VARCHAR");
-                    }
-            }
-            if (diff == ColumnDiff.LESS) {
-                try (Connection conn3 = getConnection(tenantView, tenantId)) {
-                    conn3.createStatement().execute(
-                        "ALTER VIEW " + fullViewName + " DROP COLUMN CF2.V2");
-                }
-                expectedBaseColumnCount = DIVERGED_VIEW_BASE_COLUMN_COUNT;
-            } else {
-                expectedBaseColumnCount = 6;
-            }
-
-            checkBaseColumnCount(tenantId, viewSchema, viewName, expectedBaseColumnCount);
-            checkBaseColumnCount(null, baseTableSchema, baseTableName, BASE_TABLE_BASE_COLUMN_COUNT);
-            
-            // remove base column count kv so we can check whether the upgrade code is setting the 
-            // base column count correctly.
-            removeBaseColumnCountKV(tenantId, viewSchema, viewName);
-            removeBaseColumnCountKV(null, baseTableSchema, baseTableName);
-
-            // assert that the removing base column count key value worked correctly.
-            checkBaseColumnCount(tenantId, viewSchema, viewName, 0);
-            checkBaseColumnCount(null, baseTableSchema, baseTableName, 0);
-            
-            // run upgrade
-            upgradeTo4_5_0(conn);
-
-            checkBaseColumnCount(tenantId, viewSchema, viewName, expectedBaseColumnCount);
-            checkBaseColumnCount(null, baseTableSchema, baseTableName, BASE_TABLE_BASE_COLUMN_COUNT);
-        } finally {
-            conn.close();
-        }
-    }
-
-    private static void checkBaseColumnCount(String tenantId, String schemaName, String tableName,
-            int expectedBaseColumnCount) throws Exception {
-        checkNotNull(tableName);
-        Connection conn = DriverManager.getConnection(getUrl());
-        String sql = SELECT_BASE_COLUMN_COUNT_FROM_HEADER_ROW;
-        sql =
-                String.format(sql, tenantId == null ? " IS NULL " : " = ? ",
-                    schemaName == null ? "IS NULL" : " = ? ");
-        int paramIndex = 1;
-        PreparedStatement stmt = conn.prepareStatement(sql);
-        if (tenantId != null) {
-            stmt.setString(paramIndex++, tenantId);
-        }
-        if (schemaName != null) {
-            stmt.setString(paramIndex++, schemaName);
-        }
-        stmt.setString(paramIndex, tableName);
-        ResultSet rs = stmt.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(expectedBaseColumnCount, rs.getInt(1));
-        assertFalse(rs.next());
-    }
-
-    private static void
-            removeBaseColumnCountKV(String tenantId, String schemaName, String tableName)
-                    throws Exception {
-        byte[] rowKey =
-                SchemaUtil.getTableKey(tenantId == null ? new byte[0] : Bytes.toBytes(tenantId),
-                    schemaName == null ? new byte[0] : Bytes.toBytes(schemaName),
-                    Bytes.toBytes(tableName));
-        Put viewColumnDefinitionPut = new Put(rowKey, HConstants.LATEST_TIMESTAMP);
-        viewColumnDefinitionPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-            PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES, HConstants.LATEST_TIMESTAMP, null);
-
-        try (PhoenixConnection conn =
-                (DriverManager.getConnection(getUrl())).unwrap(PhoenixConnection.class)) {
-            try (HTableInterface htable =
-                    conn.getQueryServices().getTable(
-                        Bytes.toBytes(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME))) {
-                RowMutations mutations = new RowMutations(rowKey);
-                mutations.add(viewColumnDefinitionPut);
-                htable.mutateRow(mutations);
-            }
-        }
-    }
-    
     @Test
     public void testUpgradeRequiredPreventsSQL() throws SQLException {
         String tableName = generateUniqueName();
@@ -709,8 +403,8 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
                     return true;
                 }
             };
-            try (PhoenixConnection phxConn = new PhoenixConnection(servicesWithUpgrade,
-                    conn.unwrap(PhoenixConnection.class), HConstants.LATEST_TIMESTAMP)) {
+            try (PhoenixConnection phxConn = new PhoenixConnection(conn.unwrap(PhoenixConnection.class),
+                    servicesWithUpgrade, conn.getClientInfo())) {
                 // Because upgrade is required, this SQL should fail.
                 try {
                     phxConn.createStatement().executeQuery("SELECT * FROM " + tableName);
@@ -842,7 +536,7 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testAddParentChildLinks() throws Exception {
+    public void testMoveParentChildLinks() throws Exception {
         String schema = "S_" + generateUniqueName();
         String table1 = "T_" + generateUniqueName();
         String table2 = "T_" + generateUniqueName();
@@ -882,13 +576,16 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
             Set<String> expectedChildLinkSet = getChildLinks(conn);
 
             // delete all the child links
-            conn.createStatement().execute("DELETE FROM SYSTEM.CATALOG WHERE LINK_TYPE = "
+            conn.createStatement().execute("DELETE FROM SYSTEM.CHILD_LINK WHERE LINK_TYPE = "
                     + LinkType.CHILD_TABLE.getSerializedValue());
 
             // re-create them by running the upgrade code
             PhoenixConnection phxMetaConn = metaConn.unwrap(PhoenixConnection.class);
             phxMetaConn.setRunningUpgrade(true);
+            // create the parent-> child links in SYSTEM.CATALOG
             UpgradeUtil.addParentToChildLinks(phxMetaConn);
+            // move the parent->child links to SYSTEM.CHILD_LINK
+            UpgradeUtil.moveChildLinks(phxMetaConn);
             Set<String> actualChildLinkSet = getChildLinks(conn);
 
             assertEquals("Unexpected child links", expectedChildLinkSet, actualChildLinkSet);
@@ -898,7 +595,7 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
     private Set<String> getChildLinks(Connection conn) throws SQLException {
         ResultSet rs =
                 conn.createStatement().executeQuery(
-                    "SELECT TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY FROM SYSTEM.CATALOG WHERE LINK_TYPE = "
+                    "SELECT TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY FROM SYSTEM.CHILD_LINK WHERE LINK_TYPE = "
                             + LinkType.CHILD_TABLE.getSerializedValue());
         Set<String> childLinkSet = Sets.newHashSet();
         while (rs.next()) {


[40/50] [abbrv] phoenix git commit: PHOENIX-4882 The client re-resolves the table for every projected non-indexed column when there's a local index.

Posted by ja...@apache.org.
PHOENIX-4882 The client re-resolves the table for every projected non-indexed column when there's a local index.


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

Branch: refs/heads/omid2
Commit: 7e1eabf1bc96799e4d50c0e406f4f803644a4490
Parents: 5900ec8
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Sep 6 11:22:35 2018 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Thu Sep 6 11:22:35 2018 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/compile/FromCompiler.java  | 5 +++++
 .../java/org/apache/phoenix/schema/LocalIndexDataColumnRef.java | 2 +-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e1eabf1/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 6eb5952..efc66a9 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
@@ -249,6 +249,11 @@ public class FromCompiler {
         return visitor;
     }
 
+    public static ColumnResolver getResolver(NamedTableNode tableNode, PhoenixConnection connection, boolean updateCacheImmediately) throws SQLException {
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableNode, updateCacheImmediately);
+        return visitor;
+    }
+
     public static ColumnResolver getResolver(NamedTableNode tableNode, PhoenixConnection connection, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
         SingleTableColumnResolver visitor =
                 new SingleTableColumnResolver(connection, tableNode, true, 0, udfParseNodes);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e1eabf1/phoenix-core/src/main/java/org/apache/phoenix/schema/LocalIndexDataColumnRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/LocalIndexDataColumnRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/LocalIndexDataColumnRef.java
index 270c66d..0061331 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/LocalIndexDataColumnRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/LocalIndexDataColumnRef.java
@@ -37,7 +37,7 @@ public class LocalIndexDataColumnRef extends ColumnRef {
         super(FromCompiler.getResolver(
             FACTORY.namedTable(null, TableName.create(context.getCurrentTable().getTable()
                     .getSchemaName().getString(), context.getCurrentTable().getTable()
-                    .getParentTableName().getString())), context.getConnection()).resolveTable(
+                    .getParentTableName().getString())), context.getConnection(), false).resolveTable(
             context.getCurrentTable().getTable().getSchemaName().getString(),
             context.getCurrentTable().getTable().getParentTableName().getString()), IndexUtil
                 .getDataColumnFamilyName(indexColumnName), IndexUtil


[22/50] [abbrv] phoenix git commit: PHOENIX-4834 PhoenixMetricsLog interface methods should not depend on specific logger

Posted by ja...@apache.org.
PHOENIX-4834 PhoenixMetricsLog interface methods should not depend on specific logger


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

Branch: refs/heads/omid2
Commit: 58570bbd4ea65d02c9c556e5ff9ae409c93f3fda
Parents: 7a2531d
Author: Karan Mehta <ka...@gmail.com>
Authored: Tue Aug 7 15:47:33 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Tue Aug 7 15:59:46 2018 -0700

----------------------------------------------------------------------
 .../apache/phoenix/monitoring/PhoenixMetricsIT.java    |  9 ++++-----
 .../apache/phoenix/jdbc/LoggingPhoenixConnection.java  | 11 ++++++-----
 .../apache/phoenix/jdbc/LoggingPhoenixResultSet.java   |  7 ++-----
 .../org/apache/phoenix/jdbc/PhoenixMetricsLog.java     | 13 ++++++-------
 4 files changed, 18 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/58570bbd/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
index f13391f..4c5c592 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
@@ -76,7 +76,6 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
-import org.slf4j.Logger;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
@@ -1046,25 +1045,25 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
         LoggingPhoenixConnection protectedConn =
                 new LoggingPhoenixConnection(conn, new PhoenixMetricsLog() {
                     @Override
-                    public void logOverAllReadRequestMetrics(Logger logger,
+                    public void logOverAllReadRequestMetrics(
                             Map<MetricType, Long> overAllQueryMetrics) {
                         overAllQueryMetricsMap.putAll(overAllQueryMetrics);
                     }
 
                     @Override
-                    public void logRequestReadMetrics(Logger logger,
+                    public void logRequestReadMetrics(
                             Map<String, Map<MetricType, Long>> requestReadMetrics) {
                         requestReadMetricsMap.putAll(requestReadMetrics);
                     }
 
                     @Override
-                    public void logWriteMetricsfoForMutations(Logger logger,
+                    public void logWriteMetricsfoForMutations(
                             Map<String, Map<MetricType, Long>> mutationWriteMetrics) {
                         mutationWriteMetricsMap.putAll(mutationWriteMetrics);
                     }
 
                     @Override
-                    public void logReadMetricInfoForMutationsSinceLastReset(Logger logger,
+                    public void logReadMetricInfoForMutationsSinceLastReset(
                             Map<String, Map<MetricType, Long>> mutationReadMetrics) {
                         mutationReadMetricsMap.putAll(mutationReadMetrics);
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/58570bbd/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
index e1b5dee..d98da83 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
@@ -23,12 +23,9 @@ import java.sql.SQLException;
 import java.sql.Statement;
 
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class LoggingPhoenixConnection extends DelegateConnection {
 
-    private static final Logger logger = LoggerFactory.getLogger(LoggingPhoenixResultSet.class);
     private PhoenixMetricsLog phoenixMetricsLog;
 
     public LoggingPhoenixConnection(Connection conn,
@@ -37,6 +34,10 @@ public class LoggingPhoenixConnection extends DelegateConnection {
         this.phoenixMetricsLog = phoenixMetricsLog;
     }
 
+    public PhoenixMetricsLog getPhoenixMetricsLog() {
+        return phoenixMetricsLog;
+    }
+
     @Override
     public Statement createStatement() throws SQLException {
         return new LoggingPhoenixStatement(super.createStatement(), phoenixMetricsLog);
@@ -101,8 +102,8 @@ public class LoggingPhoenixConnection extends DelegateConnection {
     @Override
     public void commit() throws SQLException {
         super.commit();
-        phoenixMetricsLog.logWriteMetricsfoForMutations(logger, PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
-        phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(logger, PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
+        phoenixMetricsLog.logWriteMetricsfoForMutations(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
+        phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
         PhoenixRuntime.resetMetrics(conn);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/58570bbd/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
index cc3b567..fbde499 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
@@ -21,12 +21,9 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class LoggingPhoenixResultSet extends DelegateResultSet {
     
-    private static final Logger logger = LoggerFactory.getLogger(LoggingPhoenixResultSet.class);
     private PhoenixMetricsLog phoenixMetricsLog;
     
     public LoggingPhoenixResultSet(ResultSet rs, PhoenixMetricsLog phoenixMetricsLog) {
@@ -36,8 +33,8 @@ public class LoggingPhoenixResultSet extends DelegateResultSet {
     
     @Override
     public void close() throws SQLException {
-        phoenixMetricsLog.logOverAllReadRequestMetrics(logger, PhoenixRuntime.getOverAllReadRequestMetricInfo(rs));
-        phoenixMetricsLog.logRequestReadMetrics(logger, PhoenixRuntime.getRequestReadMetricInfo(rs));
+        phoenixMetricsLog.logOverAllReadRequestMetrics(PhoenixRuntime.getOverAllReadRequestMetricInfo(rs));
+        phoenixMetricsLog.logRequestReadMetrics(PhoenixRuntime.getRequestReadMetricInfo(rs));
         PhoenixRuntime.resetMetrics(rs);
         super.close();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/58570bbd/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
index eac9bb8..be08e52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixMetricsLog.java
@@ -20,16 +20,15 @@ package org.apache.phoenix.jdbc;
 import java.util.Map;
 
 import org.apache.phoenix.monitoring.MetricType;
-import org.slf4j.Logger;
 
 public interface PhoenixMetricsLog {
 
-    void logOverAllReadRequestMetrics(Logger logger, Map<MetricType, Long> overAllQueryMetrics);
+    void logOverAllReadRequestMetrics(Map<MetricType, Long> overAllQueryMetrics);
 
-    void logRequestReadMetrics(Logger logger, Map<String, Map<MetricType, Long>> requestReadMetrics);
-    
-    void logWriteMetricsfoForMutations(Logger logger,  Map<String, Map<MetricType, Long>> mutationWriteMetrics);
-    
-    void logReadMetricInfoForMutationsSinceLastReset(Logger logger, Map<String, Map<MetricType, Long>> mutationReadMetrics);
+    void logRequestReadMetrics(Map<String, Map<MetricType, Long>> requestReadMetrics);
+
+    void logWriteMetricsfoForMutations(Map<String, Map<MetricType, Long>> mutationWriteMetrics);
+
+    void logReadMetricInfoForMutationsSinceLastReset(Map<String, Map<MetricType, Long>> mutationReadMetrics);
 
 }


[26/50] [abbrv] phoenix git commit: Revert "PHOENIX-4830 fix order by primary key desc"

Posted by ja...@apache.org.
Revert "PHOENIX-4830 fix order by primary key desc"

This reverts commit 59f8d0fd1a110786251dbf79c7bc743d1569b54c.


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

Branch: refs/heads/omid2
Commit: fe4c053c156fe350238a3bb42a41c701859c70e6
Parents: a2a3044
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Aug 13 10:43:20 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Mon Aug 13 10:43:20 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/OrderByIT.java   | 46 --------------------
 .../phoenix/iterate/TableResultIterator.java    | 10 +----
 .../java/org/apache/phoenix/util/ScanUtil.java  | 19 --------
 3 files changed, 2 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fe4c053c/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
index d7bbc05..578a3af 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
@@ -656,52 +656,6 @@ public class OrderByIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testOrderByDescOnPkWithSubQuery() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String ddl = "create table " + tableName + " (id bigint not null primary key, a bigint)";
-        conn.createStatement().execute(ddl);
-
-        conn.createStatement().execute("upsert into " + tableName + " values (1, 11)");
-        conn.createStatement().execute("upsert into " + tableName + " values (2, 22)");
-        conn.createStatement().execute("upsert into " + tableName + " values (3, 33)");
-        conn.createStatement().execute("upsert into " + tableName + " values (4,44)");
-
-        conn.commit();
-
-        ResultSet rs;
-        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
-        rs = stmt.executeQuery("select id from " + tableName + " where id in (select id from "
-            + tableName + ") order by id desc");
-        assertTrue(rs.next());
-        assertEquals("4", rs.getString(1));
-    }
-
-    @Test
-    public void testOrderByAscOnPkWithSubQuery() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String tableName = generateUniqueName();
-        String ddl = "create table " + tableName + " (id bigint not null primary key, a bigint)";
-        conn.createStatement().execute(ddl);
-
-        conn.createStatement().execute("upsert into " + tableName + " values (1, 11)");
-        conn.createStatement().execute("upsert into " + tableName + " values (2, 22)");
-        conn.createStatement().execute("upsert into " + tableName + " values (3, 33)");
-        conn.createStatement().execute("upsert into " + tableName + " values (4,44)");
-
-        conn.commit();
-
-        ResultSet rs;
-        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
-        rs = stmt.executeQuery("select id from " + tableName + " where id in (select id from "
-            + tableName + ") order by id");
-        assertTrue(rs.next());
-        assertEquals("1", rs.getString(1));
-    }
-
-    @Test
     public void testNullsLastWithDesc() throws Exception {
         Connection conn=null;
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fe4c053c/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
index 9a80d9f..06f612a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.iterate;
 
-import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.REVERSE_SCAN;
 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.iterate.TableResultIterator.RenewLeaseStatus.CLOSED;
@@ -54,7 +53,6 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.tuple.Tuple;
-import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
@@ -248,12 +246,8 @@ public class TableResultIterator implements ResultIterator {
             ResultIterator delegate = this.scanIterator;
             if (delegate == UNINITIALIZED_SCANNER) {
                 try {
-                    if (scan.getAttribute(REVERSE_SCAN) != null
-                        && (boolean)(PBoolean.INSTANCE.toObject(scan.getAttribute(REVERSE_SCAN)))) {
-                        ScanUtil.prepareStopRowForReverseScan(scan);
-                    }
-                    this.scanIterator = new ScanningResultIterator(htable.getScanner(scan), scan,
-                        scanMetricsHolder);
+                    this.scanIterator =
+                            new ScanningResultIterator(htable.getScanner(scan), scan, scanMetricsHolder);
                 } catch (IOException e) {
                     Closeables.closeQuietly(htable);
                     throw ServerUtil.parseServerException(e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fe4c053c/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 2dd46a6..62ecebd 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
@@ -656,25 +656,6 @@ public class ScanUtil {
     }
 
     /**
-     * HBase scan stopRow is exclusive.So we have to append trailing 0 to achieve inclusiveness.
-     * for reverse scan, need to append trailing F to stopRow.
-     * @param scan
-     */
-    public static void prepareStopRowForReverseScan(Scan scan) {
-        byte[] stopRow = scan.getStopRow();
-        if (stopRow == null) {
-            return;
-        }
-        byte[] newStopRow = new byte[stopRow.length + 1];
-        int i = 0;
-        for (byte nsr : stopRow) {
-            newStopRow[i++] = nsr;
-        }
-        newStopRow[i] = QueryConstants.DESC_SEPARATOR_BYTE;
-        scan.setStopRow(newStopRow);
-    }
-
-    /**
      * Start key and stop key of the original scan from client are regions start and end keys so
      * prefix scan start/stop key to the start row/stop row suffix and set them as scan boundaries.
      * @param scan


[23/50] [abbrv] phoenix git commit: PHOENIX-4647 Column header doesn't handle optional arguments correctly

Posted by ja...@apache.org.
PHOENIX-4647 Column header doesn't handle optional arguments correctly


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

Branch: refs/heads/omid2
Commit: 913a85576aa9d0d7903b584d46c41af16dcee3af
Parents: 58570bb
Author: Xu Cang <xc...@salesforce.com>
Authored: Mon Aug 6 23:56:00 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Aug 8 16:33:18 2018 -0700

----------------------------------------------------------------------
 .../expression/function/SubstrFunction.java     | 20 ++++++++++++++++++++
 .../phoenix/compile/WhereOptimizerTest.java     | 12 ++++++++++++
 .../java/org/apache/phoenix/util/TestUtil.java  |  5 +++++
 3 files changed, 37 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/913a8557/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
index feaa07a..0d6d1c9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
@@ -211,5 +211,25 @@ public class SubstrFunction extends PrefixFunction {
     public String getName() {
         return NAME;
     }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder(getName() + "(");
+        if (children.size()==0)
+            return buf.append(")").toString();
+        if (hasLengthExpression) {
+            buf.append(getStrExpression());
+            buf.append(", ");
+            buf.append(getOffsetExpression());
+            buf.append(", ");
+            buf.append(getLengthExpression());
+         } else {
+            buf.append(getStrExpression());
+            buf.append(", ");
+            buf.append(getOffsetExpression());
+        }
+        buf.append(")");
+        return buf.toString();
+    }
     
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/913a8557/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index e5555d6..cc6f55a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -29,6 +29,7 @@ import static org.apache.phoenix.util.TestUtil.like;
 import static org.apache.phoenix.util.TestUtil.not;
 import static org.apache.phoenix.util.TestUtil.rowKeyFilter;
 import static org.apache.phoenix.util.TestUtil.substr;
+import static org.apache.phoenix.util.TestUtil.substr2;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -62,6 +63,7 @@ import org.apache.phoenix.compile.WhereOptimizer.KeyExpressionVisitor.SingleKeyS
 import org.apache.phoenix.compile.WhereOptimizer.KeyExpressionVisitor.SlotsIterator;
 import org.apache.phoenix.compile.WhereOptimizer.KeyExpressionVisitor.TrailingRangeIterator;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.SubstrFunction;
 import org.apache.phoenix.filter.BooleanExpressionFilter;
 import org.apache.phoenix.filter.RowKeyComparisonFilter;
 import org.apache.phoenix.filter.SingleCQKeyValueComparisonFilter;
@@ -590,6 +592,16 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testSubstrExpressionWithoutLengthVariable() {
+        assertEquals("SUBSTR(ENTITY_ID, 1)",((SubstrFunction)substr2(ENTITY_ID,1)).toString());
+    }
+
+    @Test
+    public void testSubstrExpressionWithLengthVariable() {
+        assertEquals("SUBSTR(ENTITY_ID, 1, 10)",((SubstrFunction)substr(ENTITY_ID,1, 10)).toString());
+    }
+
+    @Test
     public void testTrailingSubstrExpression() throws SQLException {
         String tenantId = "0xD000000000001";
         String entityId = "002333333333333";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/913a8557/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 1683a13..0def76f 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
@@ -352,6 +352,11 @@ public class TestUtil {
         return  new SubstrFunction(Arrays.asList(e, LiteralExpression.newConstant(offset), LiteralExpression.newConstant(length)));
     }
 
+    public static Expression substr2(Expression e, Object offset) {
+
+        return  new SubstrFunction(Arrays.asList(e, LiteralExpression.newConstant(offset), LiteralExpression.newConstant(null)));
+    }
+
     public static Expression columnComparison(CompareOp op, Expression c1, Expression c2) {
         return  new ComparisonExpression(Arrays.<Expression>asList(c1, c2), op);
     }


[04/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 8dd4a88..dab1048 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
@@ -29,9 +29,10 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.ColumnProjector;
 import org.apache.phoenix.compile.ExpressionProjector;
@@ -40,7 +41,12 @@ import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.LikeExpression;
+import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.StringBasedLikeExpression;
 import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
 import org.apache.phoenix.expression.function.IndexStateNameFunction;
 import org.apache.phoenix.expression.function.SQLIndexTypeFunction;
@@ -48,25 +54,33 @@ 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.iterate.DelegateResultIterator;
 import org.apache.phoenix.iterate.MaterializedResultIterator;
 import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PSmallint;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 
@@ -336,6 +350,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final byte[] COLUMN_QUALIFIER_COUNTER_BYTES = Bytes.toBytes(COLUMN_QUALIFIER_COUNTER);
     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);
+    
+    public static final String SYSTEM_CHILD_LINK_TABLE = "CHILD_LINK";
+    public static final String SYSTEM_CHILD_LINK_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CHILD_LINK_TABLE);
+    public static final byte[] SYSTEM_CHILD_LINK_NAME_BYTES = Bytes.toBytes(SYSTEM_CHILD_LINK_NAME);
+    public static final TableName SYSTEM_LINK_HBASE_TABLE_NAME = TableName.valueOf(SYSTEM_CHILD_LINK_NAME);
 
     
     //SYSTEM:LOG
@@ -467,179 +486,352 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     private static void appendConjunction(StringBuilder buf) {
         buf.append(buf.length() == 0 ? "" : " and ");
     }
-
+    
+    private static final PColumnImpl TENANT_ID_COLUMN = new PColumnImpl(PNameFactory.newName(TENANT_ID),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, DATA_TYPE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl TABLE_SCHEM_COLUMN = new PColumnImpl(PNameFactory.newName(TABLE_SCHEM),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, DATA_TYPE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl TABLE_NAME_COLUMN = new PColumnImpl(PNameFactory.newName(TABLE_NAME),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, DATA_TYPE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl COLUMN_NAME_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_NAME),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, DATA_TYPE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(DATA_TYPE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, DATA_TYPE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl TYPE_NAME_COLUMN = new PColumnImpl(PNameFactory.newName(TYPE_NAME),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(TYPE_NAME), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl COLUMN_SIZE_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, COLUMN_SIZE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl BUFFER_LENGTH_COLUMN = new PColumnImpl(PNameFactory.newName(BUFFER_LENGTH),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(BUFFER_LENGTH), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl DECIMAL_DIGITS_COLUMN = new PColumnImpl(PNameFactory.newName(DECIMAL_DIGITS),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, DECIMAL_DIGITS_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl NUM_PREC_RADIX_COLUMN = new PColumnImpl(PNameFactory.newName(NUM_PREC_RADIX),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(NUM_PREC_RADIX), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl NULLABLE_COLUMN = new PColumnImpl(PNameFactory.newName(NULLABLE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, NULLABLE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl REMARKS_COLUMN = new PColumnImpl(PNameFactory.newName(REMARKS),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(REMARKS), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl COLUMN_DEF_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_DEF),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(COLUMN_DEF), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl SQL_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATA_TYPE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SQL_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl SQL_DATETIME_SUB_COLUMN = new PColumnImpl(PNameFactory.newName(SQL_DATETIME_SUB),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SQL_DATETIME_SUB), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl CHAR_OCTET_LENGTH_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(CHAR_OCTET_LENGTH), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl ORDINAL_POSITION_COLUMN = new PColumnImpl(PNameFactory.newName(ORDINAL_POSITION),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, ORDINAL_POSITION_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl IS_NULLABLE_COLUMN = new PColumnImpl(PNameFactory.newName(IS_NULLABLE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(IS_NULLABLE), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl SCOPE_CATALOG_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_CATALOG),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl SCOPE_SCHEMA_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_SCHEMA),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SCOPE_SCHEMA), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl SCOPE_TABLE_COLUMN = new PColumnImpl(PNameFactory.newName(SCOPE_TABLE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SCOPE_TABLE), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl SOURCE_DATA_TYPE_COLUMN = new PColumnImpl(PNameFactory.newName(SOURCE_DATA_TYPE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SOURCE_DATA_TYPE), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl IS_AUTOINCREMENT_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(SCOPE_CATALOG), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl ARRAY_SIZE_COLUMN = new PColumnImpl(PNameFactory.newName(ARRAY_SIZE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, ARRAY_SIZE_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl COLUMN_FAMILY_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_FAMILY),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarchar.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, COLUMN_FAMILY_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl TYPE_ID_COLUMN = new PColumnImpl(PNameFactory.newName(COLUMN_SIZE),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PInteger.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, Bytes.toBytes(TYPE_ID), HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl VIEW_CONSTANT_COLUMN = new PColumnImpl(PNameFactory.newName(VIEW_CONSTANT),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PVarbinary.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, VIEW_CONSTANT_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl MULTI_TENANT_COLUMN = new PColumnImpl(PNameFactory.newName(MULTI_TENANT),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PBoolean.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, MULTI_TENANT_BYTES, HConstants.LATEST_TIMESTAMP);
+    private static final PColumnImpl KEY_SEQ_COLUMN = new PColumnImpl(PNameFactory.newName(KEY_SEQ),
+            PNameFactory.newName(TABLE_FAMILY_BYTES), PSmallint.INSTANCE, null, null, false, 1, SortOrder.getDefault(),
+            0, null, false, null, false, false, KEY_SEQ_BYTES, HConstants.LATEST_TIMESTAMP);
+    
+    private static final List<PColumnImpl> PK_DATUM_LIST = Lists.newArrayList(TENANT_ID_COLUMN, TABLE_SCHEM_COLUMN, TABLE_NAME_COLUMN, COLUMN_NAME_COLUMN);
+    
+    private static final RowProjector GET_COLUMNS_ROW_PROJECTOR = new RowProjector(
+            Arrays.<ColumnProjector> asList(
+                    new ExpressionProjector(TABLE_CAT, SYSTEM_CATALOG,
+                            new RowKeyColumnExpression(TENANT_ID_COLUMN,
+                                    new RowKeyValueAccessor(PK_DATUM_LIST, 0)), false),
+                    new ExpressionProjector(TABLE_SCHEM, SYSTEM_CATALOG,
+                            new RowKeyColumnExpression(TABLE_SCHEM_COLUMN,
+                                    new RowKeyValueAccessor(PK_DATUM_LIST, 1)), false),
+                    new ExpressionProjector(TABLE_NAME, SYSTEM_CATALOG,
+                            new RowKeyColumnExpression(TABLE_NAME_COLUMN,
+                                    new RowKeyValueAccessor(PK_DATUM_LIST, 2)), false),
+                    new ExpressionProjector(COLUMN_NAME, SYSTEM_CATALOG,
+                            new RowKeyColumnExpression(COLUMN_NAME_COLUMN,
+                                    new RowKeyValueAccessor(PK_DATUM_LIST, 3)), false),
+                    new ExpressionProjector(DATA_TYPE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(DATA_TYPE_COLUMN), false),
+                    new ExpressionProjector(TYPE_NAME, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(TYPE_NAME_COLUMN), false),
+                    new ExpressionProjector(COLUMN_SIZE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(COLUMN_SIZE_COLUMN), false),
+                    new ExpressionProjector(BUFFER_LENGTH, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(BUFFER_LENGTH_COLUMN), false),
+                    new ExpressionProjector(DECIMAL_DIGITS, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(DECIMAL_DIGITS_COLUMN), false),
+                    new ExpressionProjector(NUM_PREC_RADIX, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(NUM_PREC_RADIX_COLUMN), false),
+                    new ExpressionProjector(NULLABLE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(NULLABLE_COLUMN), false),
+                    new ExpressionProjector(REMARKS, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(REMARKS_COLUMN), false),
+                    new ExpressionProjector(COLUMN_DEF, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(COLUMN_DEF_COLUMN), false),
+                    new ExpressionProjector(SQL_DATA_TYPE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(SQL_DATA_TYPE_COLUMN), false),
+                    new ExpressionProjector(SQL_DATETIME_SUB, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(SQL_DATETIME_SUB_COLUMN), false),
+                    new ExpressionProjector(CHAR_OCTET_LENGTH, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(CHAR_OCTET_LENGTH_COLUMN), false),
+                    new ExpressionProjector(ORDINAL_POSITION, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(ORDINAL_POSITION_COLUMN), false),
+                    new ExpressionProjector(IS_NULLABLE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(IS_NULLABLE_COLUMN), false),
+                    new ExpressionProjector(SCOPE_CATALOG, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(SCOPE_CATALOG_COLUMN), false),
+                    new ExpressionProjector(SCOPE_SCHEMA, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(SCOPE_SCHEMA_COLUMN), false),
+                    new ExpressionProjector(SCOPE_TABLE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(SCOPE_TABLE_COLUMN), false),
+                    new ExpressionProjector(SOURCE_DATA_TYPE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(SOURCE_DATA_TYPE_COLUMN), false),
+                    new ExpressionProjector(IS_AUTOINCREMENT, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(IS_AUTOINCREMENT_COLUMN), false),
+                    new ExpressionProjector(ARRAY_SIZE, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(ARRAY_SIZE_COLUMN), false),
+                    new ExpressionProjector(COLUMN_FAMILY, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(COLUMN_FAMILY_COLUMN), false),
+                    new ExpressionProjector(TYPE_ID, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(TYPE_ID_COLUMN), false),
+                    new ExpressionProjector(VIEW_CONSTANT, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(VIEW_CONSTANT_COLUMN), false),
+                    new ExpressionProjector(MULTI_TENANT, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(MULTI_TENANT_COLUMN), false),
+                    new ExpressionProjector(KEY_SEQ, SYSTEM_CATALOG,
+                            new KeyValueColumnExpression(KEY_SEQ_COLUMN), false)
+                    ), 0, true);
+    
+    private boolean match(String str, String pattern) throws SQLException {
+        LiteralExpression strExpr = LiteralExpression.newConstant(str, PVarchar.INSTANCE, SortOrder.ASC);
+        LiteralExpression patternExpr = LiteralExpression.newConstant(pattern, PVarchar.INSTANCE, SortOrder.ASC);
+        List<Expression> children = Arrays.<Expression>asList(strExpr, patternExpr);
+        LikeExpression likeExpr = StringBasedLikeExpression.create(children, LikeType.CASE_SENSITIVE);
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        boolean evaluated = likeExpr.evaluate(null, ptr);
+        Boolean result = (Boolean)likeExpr.getDataType().toObject(ptr);
+        if (evaluated) {
+            return result;
+        }
+        return false;
+    }
+    
     @Override
     public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
             throws SQLException {
-        StringBuilder buf = new StringBuilder("select \n " +
-                TENANT_ID + " " + TABLE_CAT + "," + // use this for tenant id
-                TABLE_SCHEM + "," +
-                TABLE_NAME + " ," +
-                COLUMN_NAME + "," +
-                ExternalSqlTypeIdFunction.NAME + "(" + DATA_TYPE + ") AS " + DATA_TYPE + "," +
-                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," +
-                COLUMN_SIZE + "," +
-                BUFFER_LENGTH + "," +
-                DECIMAL_DIGITS + "," +
-                NUM_PREC_RADIX + "," +
-                NULLABLE + "," +
-                REMARKS + "," +
-                COLUMN_DEF + "," +
-                SQL_DATA_TYPE + "," +
-                SQL_DATETIME_SUB + "," +
-                CHAR_OCTET_LENGTH + "," +
-                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + ORDINAL_POSITION + "-1 ELSE " + ORDINAL_POSITION + " END AS " + ORDINAL_POSITION + "," +
-                "CASE " + NULLABLE + " WHEN " + DatabaseMetaData.attributeNoNulls +  " THEN '" + Boolean.FALSE.toString() + "' WHEN " + DatabaseMetaData.attributeNullable + " THEN '" + Boolean.TRUE.toString() + "' END AS " + IS_NULLABLE + "," +
-                SCOPE_CATALOG + "," +
-                SCOPE_SCHEMA + "," +
-                SCOPE_TABLE + "," +
-                SOURCE_DATA_TYPE + "," +
-                IS_AUTOINCREMENT + "," +
-                ARRAY_SIZE + "," +
-                COLUMN_FAMILY + "," +
-                DATA_TYPE + " " + TYPE_ID + "," +// raw type id for potential internal consumption
-                VIEW_CONSTANT + "," +
-                MULTI_TENANT + "," +
-                "CASE WHEN " + TENANT_POS_SHIFT + " THEN " + KEY_SEQ + "-1 ELSE " + KEY_SEQ + " END AS " + KEY_SEQ +
-                " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS + "(" + TENANT_POS_SHIFT + " BOOLEAN)");
-        StringBuilder where = new StringBuilder();
-        addTenantIdFilter(where, catalog);
-        if (schemaPattern != null) {
-            appendConjunction(where);
-            where.append(TABLE_SCHEM + (schemaPattern.length() == 0 ? " is null" : " like '" + StringUtil.escapeStringConstant(schemaPattern) + "'" ));
-        }
-        if (tableNamePattern != null && tableNamePattern.length() > 0) {
-            appendConjunction(where);
-            where.append(TABLE_NAME + " like '" + StringUtil.escapeStringConstant(tableNamePattern) + "'" );
-        }
-        // Allow a "." in columnNamePattern for column family match
-        String colPattern = null;
-        if (columnNamePattern != null && columnNamePattern.length() > 0) {
+        boolean isTenantSpecificConnection = connection.getTenantId() != null;
+        List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
+        ResultSet rs = getTables(catalog, schemaPattern, tableNamePattern, null);
+        while (rs.next()) {
+            String schemaName = rs.getString(TABLE_SCHEM);
+            // Allow a "." in columnNamePattern for column family match
+            String colPattern = null;
             String cfPattern = null;
-            int index = columnNamePattern.indexOf('.');
-            if (index <= 0) {
-                colPattern = columnNamePattern;
-            } else {
-                cfPattern = columnNamePattern.substring(0, index);
-                if (columnNamePattern.length() > index+1) {
-                    colPattern = columnNamePattern.substring(index+1);
+            if (columnNamePattern != null && columnNamePattern.length() > 0) {
+                int index = columnNamePattern.indexOf('.');
+                if (index <= 0) {
+                    colPattern = columnNamePattern;
+                } else {
+                    cfPattern = columnNamePattern.substring(0, index);
+                    if (columnNamePattern.length() > index+1) {
+                        colPattern = columnNamePattern.substring(index+1);
+                    }
                 }
             }
-            if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
-                // Will pick up only KV columns
-                appendConjunction(where);
-                where.append(COLUMN_FAMILY + " like '" + StringUtil.escapeStringConstant(cfPattern) + "'" );
-            }
-            if (colPattern != null && colPattern.length() > 0) {
-                appendConjunction(where);
-                where.append(COLUMN_NAME + " like '" + StringUtil.escapeStringConstant(colPattern) + "'" );
-            }
-        }
-        if (colPattern == null || colPattern.length() == 0) {
-            appendConjunction(where);
-            where.append(COLUMN_NAME + " is not null" );
-            appendConjunction(where);
-            where.append(LINK_TYPE + " is null" );
-        }
-        boolean isTenantSpecificConnection = connection.getTenantId() != null;
-        if (isTenantSpecificConnection) {
-            buf.append(" where (" + where + ") OR ("
-                    + COLUMN_FAMILY + " is null AND " +  COLUMN_NAME + " is null)");
-        } else {
-            buf.append(" where " + where);
-        }
-        buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + "," + SYSTEM_CATALOG_ALIAS + "." + ORDINAL_POSITION);
-
-        Statement stmt;
-        if (isTenantSpecificConnection) {
-            stmt = connection.createStatement(new PhoenixStatementFactory() {
-                @Override
-                public PhoenixStatement newStatement(PhoenixConnection connection) {
-                    return new PhoenixStatement(connection) {
-                        @Override
-                        public PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector,
-                                StatementContext context) throws SQLException {
-                            return new PhoenixResultSet(new TenantColumnFilteringIterator(iterator, projector),
-                                    projector, context);
-                        }
-                    };
+            String tableName = rs.getString(TABLE_NAME);
+            String tenantId = rs.getString(TABLE_CAT);
+            String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+            PTable table = PhoenixRuntime.getTable(connection, fullTableName);
+            boolean isSalted = table.getBucketNum()!=null;
+            boolean tenantColSkipped = false;
+            for (PColumn column : table.getColumns()) {
+                if (isTenantSpecificConnection && column.equals(table.getPKColumns().get(0))) {
+                    // skip the tenant column
+                    tenantColSkipped = true;
+                    continue;
                 }
-            });
-        } else {
-            stmt = connection.createStatement();
-        }
-        return stmt.executeQuery(buf.toString());
-    }
-    
-//    private ColumnResolver getColumnResolverForCatalogTable() throws SQLException {
-//        TableRef tableRef = new TableRef(getTable(connection, SYSTEM_CATALOG_NAME));
-//        return FromCompiler.getResolver(tableRef);
-//    }
-    
-    /**
-     * Filters the tenant id column out of a column metadata result set (thus, where each row is a column definition).
-     * The tenant id is by definition the first column of the primary key, but the primary key does not necessarily
-     * start at the first column. Assumes columns are sorted on ordinal position.
-     */
-    private static class TenantColumnFilteringIterator extends DelegateResultIterator {
-        private final RowProjector rowProjector;
-        private final int columnFamilyIndex;
-        private final int columnNameIndex;
-        private final int multiTenantIndex;
-        private final int keySeqIndex;
-        private boolean inMultiTenantTable;
-        private boolean tenantColumnSkipped;
-
-        private TenantColumnFilteringIterator(ResultIterator delegate, RowProjector rowProjector) throws SQLException {
-            super(delegate);
-            this.rowProjector = rowProjector;
-            this.columnFamilyIndex = rowProjector.getColumnIndex(COLUMN_FAMILY);
-            this.columnNameIndex = rowProjector.getColumnIndex(COLUMN_NAME);
-            this.multiTenantIndex = rowProjector.getColumnIndex(MULTI_TENANT);
-            this.keySeqIndex = rowProjector.getColumnIndex(KEY_SEQ);
-        }
-
-        @Override
-        public Tuple next() throws SQLException {
-            Tuple tuple = super.next();
-
-            while (tuple != null
-                    && getColumn(tuple, columnFamilyIndex) == null && getColumn(tuple, columnNameIndex) == null) {
-                // new table, check if it is multitenant
-                inMultiTenantTable = getColumn(tuple, multiTenantIndex) == Boolean.TRUE;
-                tenantColumnSkipped = false;
-                // skip row representing table
-                tuple = super.next();
-            }
-
-            if (tuple != null && inMultiTenantTable && !tenantColumnSkipped) {
-                Object value = getColumn(tuple, keySeqIndex);
-                if (value != null && ((Number)value).longValue() == 1L) {
-                    tenantColumnSkipped = true;
-                    // skip tenant id primary key column
-                    return next();
+                String columnFamily = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
+                String columnName = column.getName().getString();
+                if (cfPattern != null && cfPattern.length() > 0) { // if null or empty, will pick up all columns
+                    if (columnFamily==null || !match(columnFamily, cfPattern)) {
+                        continue;
+                    }
                 }
+                if (colPattern != null && colPattern.length() > 0) {
+                    if (!match(columnName, colPattern)) {
+                        continue;
+                    }
+                }
+                // generate row key
+                // TENANT_ID, TABLE_SCHEM, TABLE_NAME , COLUMN_NAME are row key columns
+                byte[] rowKey =
+                        SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName, null);
+
+                // add one cell for each column info
+                List<Cell> cells = Lists.newArrayListWithCapacity(25);
+                // DATA_TYPE
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES,
+                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    PInteger.INSTANCE.toBytes(column.getDataType().getResultSetSqlType())));
+                // TYPE_NAME
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(TYPE_NAME), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    column.getDataType().getSqlTypeNameBytes()));
+                // COLUMN_SIZE
+                cells.add(
+                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, COLUMN_SIZE_BYTES,
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                        column.getMaxLength() != null
+                                ? PInteger.INSTANCE.toBytes(column.getMaxLength())
+                                : ByteUtil.EMPTY_BYTE_ARRAY));
+                // BUFFER_LENGTH
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(BUFFER_LENGTH), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // DECIMAL_DIGITS
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES,
+                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    column.getScale() != null ? PInteger.INSTANCE.toBytes(column.getScale())
+                            : ByteUtil.EMPTY_BYTE_ARRAY));
+                // NUM_PREC_RADIX
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(NUM_PREC_RADIX), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // NULLABLE
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, NULLABLE_BYTES,
+                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    PInteger.INSTANCE.toBytes(SchemaUtil.getIsNullableInt(column.isNullable()))));
+                // REMARKS
+                cells.add(
+                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, Bytes.toBytes(REMARKS),
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY));
+                // COLUMN_DEF
+                cells.add(
+                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, Bytes.toBytes(COLUMN_DEF),
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY));
+                // SQL_DATA_TYPE
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(SQL_DATA_TYPE), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // SQL_DATETIME_SUB
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(SQL_DATETIME_SUB), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // CHAR_OCTET_LENGTH
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(CHAR_OCTET_LENGTH), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // ORDINAL_POSITION
+                int ordinal =
+                        column.getPosition() + (isSalted ? 0 : 1) - (tenantColSkipped ? 1 : 0);
+                cells.add(
+                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, ORDINAL_POSITION_BYTES,
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, PInteger.INSTANCE.toBytes(ordinal)));
+                String isNullable =
+                        column.isNullable() ? Boolean.TRUE.toString() : Boolean.FALSE.toString();
+                // IS_NULLABLE
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(IS_NULLABLE), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    PVarchar.INSTANCE.toBytes(isNullable)));
+                // SCOPE_CATALOG
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(SCOPE_CATALOG), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // SCOPE_SCHEMA
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(SCOPE_SCHEMA), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // SCOPE_TABLE
+                cells.add(
+                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, Bytes.toBytes(SCOPE_TABLE),
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY));
+                // SOURCE_DATA_TYPE
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(SOURCE_DATA_TYPE), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // IS_AUTOINCREMENT
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(IS_AUTOINCREMENT), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    ByteUtil.EMPTY_BYTE_ARRAY));
+                // ARRAY_SIZE
+                cells.add(
+                    KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, ARRAY_SIZE_BYTES,
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                        column.getArraySize() != null
+                                ? PInteger.INSTANCE.toBytes(column.getArraySize())
+                                : ByteUtil.EMPTY_BYTE_ARRAY));
+                // COLUMN_FAMILY
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, COLUMN_FAMILY_BYTES,
+                    MetaDataProtocol.MIN_TABLE_TIMESTAMP, column.getFamilyName() != null
+                            ? column.getFamilyName().getBytes() : ByteUtil.EMPTY_BYTE_ARRAY));
+                // TYPE_ID
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES,
+                    Bytes.toBytes(TYPE_ID), MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    PInteger.INSTANCE.toBytes(column.getDataType().getSqlType())));
+                // VIEW_CONSTANT
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, VIEW_CONSTANT_BYTES,
+                    MetaDataProtocol.MIN_TABLE_TIMESTAMP, column.getViewConstant() != null
+                            ? column.getViewConstant() : ByteUtil.EMPTY_BYTE_ARRAY));
+                // MULTI_TENANT
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, MULTI_TENANT_BYTES,
+                    MetaDataProtocol.MIN_TABLE_TIMESTAMP,
+                    PBoolean.INSTANCE.toBytes(table.isMultiTenant())));
+                // KEY_SEQ_COLUMN
+                byte[] keySeqBytes = ByteUtil.EMPTY_BYTE_ARRAY;
+                int pkPos = table.getPKColumns().indexOf(column);
+                if (pkPos!=-1) {
+                    short keySeq = (short) (pkPos + 1 - (isSalted ? 1 : 0) - (tenantColSkipped ? 1 : 0));
+                    keySeqBytes = PSmallint.INSTANCE.toBytes(keySeq);
+                }
+                cells.add(KeyValueUtil.newKeyValue(rowKey, TABLE_FAMILY_BYTES, KEY_SEQ_BYTES,
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, keySeqBytes));
+                Collections.sort(cells, new CellComparator());
+                Tuple tuple = new MultiKeyValueTuple(cells);
+                tuples.add(tuple);
             }
-
-            if (tuple != null && tenantColumnSkipped) {
-                ResultTuple resultTuple = (ResultTuple)tuple;
-                List<Cell> cells = resultTuple.getResult().listCells();
-                KeyValue kv = new KeyValue(resultTuple.getResult().getRow(), TABLE_FAMILY_BYTES,
-                        TENANT_POS_SHIFT_BYTES, PDataType.TRUE_BYTES);
-                List<Cell> newCells = Lists.newArrayListWithCapacity(cells.size() + 1);
-                newCells.addAll(cells);
-                newCells.add(kv);
-                Collections.sort(newCells, KeyValue.COMPARATOR);
-                tuple = new ResultTuple(Result.create(newCells));
-            }
-            return tuple;
-        }
-
-        private Object getColumn(Tuple tuple, int index) throws SQLException {
-            ColumnProjector projector = this.rowProjector.getColumnProjector(index);
-            PDataType type = projector.getExpression().getDataType();
-            return projector.getValue(tuple, type, new ImmutableBytesPtr());
         }
+        return new PhoenixResultSet(new MaterializedResultIterator(tuples), GET_COLUMNS_ROW_PROJECTOR, new StatementContext(new PhoenixStatement(connection), false));
     }
 
     @Override
@@ -992,6 +1184,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     }
 
     @Override
+    // TODO does this need to change to use the PARENT_TABLE link
     public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern) throws SQLException {
         StringBuilder buf = new StringBuilder("select \n" +
                 TENANT_ID + " " + TABLE_CAT + "," + // Use tenantId for catalog
@@ -1028,11 +1221,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
             throws SQLException {
         return emptyResultSet;
     }
-
+    
     private static final PDatum TABLE_TYPE_DATUM = new PDatum() {
         @Override
         public boolean isNullable() {
-            return false;
+            return true;
         }
         @Override
         public PDataType getDataType() {
@@ -1051,6 +1244,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
             return SortOrder.getDefault();
         }
     };
+
     private static final RowProjector TABLE_TYPE_ROW_PROJECTOR = new RowProjector(Arrays.<ColumnProjector>asList(
             new ExpressionProjector(TABLE_TYPE, SYSTEM_CATALOG,
                     new RowKeyColumnExpression(TABLE_TYPE_DATUM,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 015f04c..048ff81 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
@@ -528,7 +528,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
             new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_BYTES_ESTIMATE),
                     PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, true, 1,
                     SortOrder.getDefault(), 0, null, false, null, false, false,
-                    EXPLAIN_PLAN_BYTES_ESTIMATE);
+                    EXPLAIN_PLAN_BYTES_ESTIMATE, 0, false);
 
     private static final String EXPLAIN_PLAN_ROWS_ESTIMATE_COLUMN_NAME = "RowsEstimate";
     private static final byte[] EXPLAIN_PLAN_ROWS_ESTIMATE =
@@ -538,7 +538,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
             new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_ROWS_ESTIMATE),
                     PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, true, 2,
                     SortOrder.getDefault(), 0, null, false, null, false, false,
-                    EXPLAIN_PLAN_ROWS_ESTIMATE);
+                    EXPLAIN_PLAN_ROWS_ESTIMATE, 0, false);
 
     private static final String EXPLAIN_PLAN_ESTIMATE_INFO_TS_COLUMN_NAME = "EstimateInfoTS";
     private static final byte[] EXPLAIN_PLAN_ESTIMATE_INFO_TS =
@@ -548,7 +548,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
             new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_ESTIMATE_INFO_TS),
                 PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, null, null, true, 3,
                 SortOrder.getDefault(), 0, null, false, null, false, false,
-                EXPLAIN_PLAN_ESTIMATE_INFO_TS);
+                EXPLAIN_PLAN_ESTIMATE_INFO_TS, 0, false);
 
     private static final RowProjector EXPLAIN_PLAN_ROW_PROJECTOR_WITH_BYTE_ROW_ESTIMATES =
             new RowProjector(Arrays
@@ -1109,7 +1109,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
     private static class ExecutableDropTableStatement extends DropTableStatement implements CompilableStatement {
 
         ExecutableDropTableStatement(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) {
-            super(tableName, tableType, ifExists, cascade);
+            super(tableName, tableType, ifExists, cascade, false);
         }
 
         @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
index 997b695..c334a81 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
@@ -25,13 +25,15 @@ public class DropTableStatement extends MutableStatement {
     private final boolean ifExists;
     private final PTableType tableType;
     private final boolean cascade;
+    private final boolean skipAddingParentColumns;
     
 
-    protected DropTableStatement(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) {
+    public DropTableStatement(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade, boolean skipAddingParentColumns) {
         this.tableName = tableName;
         this.tableType = tableType;
         this.ifExists = ifExists;
         this.cascade = cascade;
+        this.skipAddingParentColumns = skipAddingParentColumns;
     }
     
     @Override
@@ -59,4 +61,8 @@ public class DropTableStatement extends MutableStatement {
     public Operation getOperation() {
         return Operation.DELETE;
     }
+
+	public boolean getSkipAddingParentColumns() {
+		return skipAddingParentColumns;
+	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 9be59f3..aef2a84 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -366,7 +366,7 @@ public class ParseNodeFactory {
     }
 
     public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) {
-        return new DropTableStatement(tableName, tableType, ifExists, cascade);
+        return new DropTableStatement(tableName, tableType, ifExists, cascade, false);
     }
 
     public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 092bfe9..6f8cbc0 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
@@ -79,13 +79,26 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
 
     public PhoenixConnection connect(String url, Properties info) throws SQLException;
 
-    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaName, byte[] tableName, long tableTimestamp, long clientTimetamp) throws SQLException;
+    /**
+     * @param tableTimestamp timestamp of table if its present in the client side cache
+     * @param clientTimetamp if the client connection has an scn, or of the table is transactional
+     *            the txn write pointer
+     * @param skipAddingIndexes if true will the returned PTable will not include any indexes
+     * @param skipAddingParentColumns if true will the returned PTable will not include any columns
+     *            derived from ancestors
+     * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to
+     *            resolve this table as its locked)
+     * @return PTable for the given tenant id, schema and table name
+     */
+    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaName, byte[] tableName,
+            long tableTimestamp, long clientTimetamp, boolean skipAddingIndexes,
+            boolean skipAddingParentColumns, PTable lockedAncestorTable) throws SQLException;
     public MetaDataMutationResult getFunctions(PName tenantId, List<Pair<byte[], Long>> functionNameAndTimeStampPairs, long clientTimestamp) throws SQLException;
 
     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, boolean isDoNotUpgradePropSet) throws SQLException;
-    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException;
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade, boolean skipAddingParentColumns) 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;
     public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 d3cad64..4c7630d 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
@@ -58,7 +58,9 @@ import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THRESHOLD_MILLISECONDS;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS;
 import static org.apache.phoenix.util.UpgradeUtil.addParentToChildLinks;
+import static org.apache.phoenix.util.UpgradeUtil.addViewIndexToParentLinks;
 import static org.apache.phoenix.util.UpgradeUtil.getSysCatalogSnapshotName;
+import static org.apache.phoenix.util.UpgradeUtil.moveChildLinks;
 import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
 
 import java.io.IOException;
@@ -206,6 +208,7 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PSynchronizedMetaData;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
@@ -1553,8 +1556,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public MetaDataMutationResult getTable(final PName tenantId, final byte[] schemaBytes, final byte[] tableBytes,
-            final long tableTimestamp, final long clientTimestamp) throws SQLException {
+    public MetaDataMutationResult getTable(final PName tenantId, final byte[] schemaBytes,
+            final byte[] tableBytes, final long tableTimestamp, final long clientTimestamp,
+            final boolean skipAddingIndexes, final boolean skipAddingParentColumns,
+            final PTable lockedAncestorTable) throws SQLException {
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
@@ -1571,6 +1576,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 builder.setTableTimestamp(tableTimestamp);
                 builder.setClientTimestamp(clientTimestamp);
                 builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                builder.setSkipAddingParentColumns(skipAddingParentColumns);
+                builder.setSkipAddingIndexes(skipAddingIndexes);
+                if (lockedAncestorTable!=null)
+                    builder.setLockedAncestorTable(PTableImpl.toProto(lockedAncestorTable));
                 instance.getTable(controller, builder.build(), rpcCallback);
                 if(controller.getFailedOn() != null) {
                     throw controller.getFailedOn();
@@ -1582,7 +1591,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     @Override
     public MetaDataMutationResult dropTable(final List<Mutation> tableMetaData, final PTableType tableType,
-            final boolean cascade) throws SQLException {
+            final boolean cascade, final boolean skipAddingParentColumns) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
@@ -1604,6 +1613,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 builder.setTableType(tableType.getSerializedValue());
                 builder.setCascade(cascade);
                 builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                builder.setSkipAddingParentColumns(skipAddingParentColumns);
                 instance.dropTable(controller, builder.build(), rpcCallback);
                 if(controller.getFailedOn() != null) {
                     throw controller.getFailedOn();
@@ -1761,7 +1771,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaName = Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(fullTableName));
             byte[] tableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(fullTableName));
             MetaDataMutationResult result = this.getTable(tenantId, schemaName, tableName, HConstants.LATEST_TIMESTAMP,
-                    timestamp);
+                    timestamp, false, false, null);
             table = result.getTable();
             if (table == null) { throw e; }
         }
@@ -2447,6 +2457,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP;
     }
     
+    
     // Available for testing
     protected void setUpgradeRequired() {
         this.upgradeRequired.set(true);
@@ -2486,6 +2497,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return String.format(ddl, props.getInt(LOG_SALT_BUCKETS_ATTRIB, QueryServicesOptions.DEFAULT_LOG_SALT_BUCKETS));
 
     }
+    
+    // Available for testing
+    protected String getChildLinkDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_CHILD_LINK_METADATA);
+    }
 
     private String setSystemDDLProperties(String ddl) {
         return String.format(ddl,
@@ -2704,6 +2720,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.createStatement().execute(getLogTableDDL());
         } catch (TableAlreadyExistsException ignore) {}
+        try {
+            metaConnection.createStatement().executeUpdate(getChildLinkDDL());
+        } catch (TableAlreadyExistsException e) {}
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
             createSysMutexTableIfNotExists(hbaseAdmin);
@@ -2984,6 +3003,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     HTableDescriptor.SPLIT_POLICY + "='" + SystemStatsSplitPolicy.class.getName() +"'"
                     );
         }
+        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0) {
+            addViewIndexToParentLinks(metaConnection);
+            moveChildLinks(metaConnection);
+        }
         return metaConnection;
     }
 
@@ -3147,6 +3170,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 metaConnection.createStatement().executeUpdate(getLogTableDDL());
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
+            try {
+                metaConnection.createStatement().executeUpdate(getChildLinkDDL());
+            } 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
@@ -3223,8 +3249,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         metaConnection.rollback();
         PColumn column = new PColumnImpl(PNameFactory.newName("COLUMN_QUALIFIER"),
                 PNameFactory.newName(DEFAULT_COLUMN_FAMILY_NAME), PVarbinary.INSTANCE, null, null, true, numColumns,
-                SortOrder.ASC, null, null, false, null, false, false,
-                Bytes.toBytes("COLUMN_QUALIFIER"));
+                SortOrder.ASC, null, null, false, null, false, false, 
+                Bytes.toBytes("COLUMN_QUALIFIER"), timestamp);
         String upsertColumnMetadata = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                 TENANT_ID + "," +
                 TABLE_SCHEM + "," +
@@ -3765,7 +3791,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public MetaDataMutationResult updateIndexState(final List<Mutation> tableMetaData, String parentTableName) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
-        byte[] tableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX], rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]);
+        byte[] tableKey =
+                SchemaUtil.getTableKey(rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX],
+                    rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX],
+                    rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]);
         return metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 94a5257..5a46214 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
@@ -176,6 +176,10 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         return String.format(ddl, props.getInt(LOG_SALT_BUCKETS_ATTRIB, QueryServicesOptions.DEFAULT_LOG_SALT_BUCKETS));
 
     }
+    
+    protected String getChildLinkDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_CHILD_LINK_METADATA);
+    }
 
     private String setSystemDDLProperties(String ddl) {
         return String.format(ddl,
@@ -233,7 +237,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp) throws SQLException {
+    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp, boolean skipAddingIndexes, boolean skipCombiningColumns, PTable ancestorTable) throws SQLException {
         // Return result that will cause client to use it's own metadata instead of needing
         // to get anything from the server (since we don't have a connection)
         try {
@@ -294,7 +298,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException {
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
         byte[] tableName = getTableName(tableMetadata, null);
         tableSplits.remove(Bytes.toString(tableName));
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
@@ -370,6 +374,11 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                 try {
                     metaConnection.createStatement().executeUpdate(getLogTableDDL());
                 } catch (NewerTableAlreadyExistsException ignore) {}
+                try {
+                    metaConnection.createStatement()
+                            .executeUpdate(getChildLinkDDL());
+                } catch (NewerTableAlreadyExistsException ignore) {
+                }
             } catch (SQLException e) {
                 sqlE = e;
             } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 cb7ce58..b3e2cb2 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
@@ -107,8 +107,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp) throws SQLException {
-        return getDelegate().getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, clientTimestamp);
+    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp, boolean skipAddingIndexes, boolean skipAddingParentColumns, PTable ancestorTable) throws SQLException {
+        return getDelegate().getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, clientTimestamp, skipAddingIndexes, skipAddingParentColumns, ancestorTable);
     }
 
     @Override
@@ -120,8 +120,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType, boolean cascade) throws SQLException {
-        return getDelegate().dropTable(tabeMetaData, tableType, cascade);
+    public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
+        return getDelegate().dropTable(tabeMetaData, tableType, cascade, skipAddingParentColumns);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 b31175a..8d8d47f 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
@@ -344,4 +344,16 @@ public interface QueryConstants {
     public static final String HASH_JOIN_CACHE_RETRIES = "hashjoin.client.retries.number";
     public static final int DEFAULT_HASH_JOIN_CACHE_RETRIES = 5;
     
-}
+	// Links from parent to child views are stored in a separate table for
+	// scalability
+	public static final String CREATE_CHILD_LINK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
+			+ SYSTEM_CHILD_LINK_TABLE + "\"(\n" +
+			// PK columns
+			TENANT_ID + " VARCHAR NULL," + TABLE_SCHEM + " VARCHAR NULL," + TABLE_NAME + " VARCHAR NOT NULL,"
+			+ COLUMN_NAME + " VARCHAR NULL," + COLUMN_FAMILY + " VARCHAR NULL," + LINK_TYPE + " UNSIGNED_TINYINT,\n"
+			+ "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME
+			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
+			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
+			+ Boolean.FALSE;
+    
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 559d165..2bb9350 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
@@ -313,6 +313,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String LOG_BUFFER_WAIT_STRATEGY = "phoenix.log.wait.strategy";
     public static final String LOG_SAMPLE_RATE = "phoenix.log.sample.rate";
 
+	public static final String SYSTEM_CATALOG_SPLITTABLE = "phoenix.system.catalog.splittable";
+
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 307c5dd..4be8f81 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
@@ -359,6 +359,8 @@ public class QueryServicesOptions {
     public static final String DEFAULT_LOG_SAMPLE_RATE = "1.0";
     public static final int DEFAULT_LOG_SALT_BUCKETS = 32;
 
+	public static final boolean DEFAULT_SYSTEM_CATALOG_SPLITTABLE = true;
+
     private final Configuration config;
 
     private QueryServicesOptions(Configuration config) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
index d1b71ef..5452298 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
@@ -22,9 +22,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.SchemaUtil;
 
 import java.util.List;
@@ -34,21 +32,17 @@ import java.util.List;
  * may change between the source and target clusters at different times, in particular
  * during cluster upgrades. However, tenant-owned data such as tenant-owned views need to
  * be copied. This WALEntryFilter will only allow tenant-owned rows in SYSTEM.CATALOG to
- * be replicated. Data from all other tables is automatically passed. It will also copy
- * child links in SYSTEM.CATALOG that are globally-owned but point to tenant-owned views.
- *
+ * be replicated. Data from all other tables is automatically passed.
  */
 public class SystemCatalogWALEntryFilter implements WALEntryFilter {
 
-  private static byte[] CHILD_TABLE_BYTES =
-      new byte[]{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
   @Override
   public WAL.Entry filter(WAL.Entry entry) {
 
-    //if the WAL.Entry's table isn't System.Catalog, it auto-passes this filter
+    //if the WAL.Entry's table isn't System.Catalog or System.Child_Link, it auto-passes this filter
     //TODO: when Phoenix drops support for pre-1.3 versions of HBase, redo as a WALCellFilter
-    if (!SchemaUtil.isMetaTable(entry.getKey().getTablename().getName())){
+    byte[] tableName = entry.getKey().getTablename().getName();
+	if (!SchemaUtil.isMetaTable(tableName)){
       return entry;
     }
 
@@ -71,35 +65,6 @@ public class SystemCatalogWALEntryFilter implements WALEntryFilter {
     ImmutableBytesWritable key =
         new ImmutableBytesWritable(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
     //rows in system.catalog that aren't tenant-owned will have a leading separator byte
-    boolean isTenantRowCell = key.get()[key.getOffset()] != QueryConstants.SEPARATOR_BYTE;
-
-    /* In addition to the tenant view rows, there are parent-child links (see PHOENIX-2051) that
-     * provide an efficient way for a parent table or view to look up its children.
-     * These rows override SYSTEM_CATALOG.COLUMN_NAME with the child tenant_id,
-     * if any, and contain only a single Cell, LINK_TYPE, which is of PTable.LinkType.Child
-     */
-    boolean isChildLinkToTenantView = false;
-    if (!isTenantRowCell) {
-      ImmutableBytesWritable columnQualifier = new ImmutableBytesWritable(cell.getQualifierArray(),
-          cell.getQualifierOffset(), cell.getQualifierLength());
-      boolean isChildLink = columnQualifier.compareTo(PhoenixDatabaseMetaData.LINK_TYPE_BYTES) == 0;
-      if (isChildLink) {
-        ImmutableBytesWritable columnValue = new ImmutableBytesWritable(cell.getValueArray(),
-            cell.getValueOffset(), cell.getValueLength());
-        if (columnValue.compareTo(CHILD_TABLE_BYTES) == 0) {
-          byte[][] rowViewKeyMetadata = new byte[5][];
-          SchemaUtil.getVarChars(key.get(), key.getOffset(),
-              key.getLength(), 0, rowViewKeyMetadata);
-          //if the child link is to a tenant-owned view,
-          // the COLUMN_NAME field will be the byte[] of the tenant
-          //otherwise, it will be an empty byte array
-          // (NOT QueryConstants.SEPARATOR_BYTE, but a byte[0])
-          isChildLinkToTenantView =
-              rowViewKeyMetadata[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length != 0;
-        }
-      }
-
-    }
-    return isTenantRowCell || isChildLinkToTenantView;
+    return key.get()[key.getOffset()] != QueryConstants.SEPARATOR_BYTE;
   }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
index 5c9cc2e..4d9abaf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
@@ -76,6 +76,21 @@ public class DelegateColumn extends DelegateDatum implements PColumn {
     }
 
     @Override
+    public long getTimestamp() {
+        return getDelegate().getTimestamp();
+    }
+
+    @Override
+    public boolean isDerived() {
+        return getDelegate().isDerived();
+    }
+
+    @Override
+    public boolean isExcluded() {
+        return getDelegate().isExcluded();
+    }
+
+    @Override
     public boolean isRowTimestamp() {
         return getDelegate().isRowTimestamp();
     }    

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 21391f3..625d03f 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
@@ -111,7 +111,6 @@ import java.io.IOException;
 import java.sql.Date;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Types;
@@ -319,7 +318,8 @@ public class MetaDataClient {
                     TABLE_TYPE +
                     ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     
-    private static final String CREATE_VIEW_LINK =
+    
+    public static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
@@ -347,6 +347,16 @@ public class MetaDataClient {
                     COLUMN_FAMILY + "," +
                     LINK_TYPE + 
                     ") VALUES (?, ?, ?, ?, ?, ?)";
+    
+    private static final String CREATE_VIEW_INDEX_PARENT_LINK =
+    		"UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + 
+                    ") VALUES (?, ?, ?, ?, ?)";
+    
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
@@ -642,7 +652,7 @@ public class MetaDataClient {
                 ConnectionQueryServices queryServices = connection.getQueryServices();
                 result =
                         queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp,
-                            resolvedTimestamp);
+                            resolvedTimestamp, false, false, null);
                 // 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
@@ -655,7 +665,7 @@ public class MetaDataClient {
                     if (result.getTable().getTimeStamp() >= resolveTimestamp) {
                         result =
                                 queryServices.getTable(tenantId, schemaBytes, tableBytes,
-                                    tableTimestamp, resolveTimestamp);
+                                    tableTimestamp, resolveTimestamp, false, false, null);
                     }
                 }
 
@@ -934,7 +944,7 @@ public class MetaDataClient {
         colUpsert.setString(4, column.getName().getString());
         colUpsert.setString(5, column.getFamilyName() == null ? null : column.getFamilyName().getString());
         colUpsert.setInt(6, column.getDataType().getSqlType());
-        colUpsert.setInt(7, column.isNullable() ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls);
+        colUpsert.setInt(7, SchemaUtil.getIsNullableInt(column.isNullable()));
         if (column.getMaxLength() == null) {
             colUpsert.setNull(8, Types.INTEGER);
         } else {
@@ -979,7 +989,7 @@ public class MetaDataClient {
         colUpsert.execute();
     }
 
-    private void addFunctionArgMutation(String functionName, FunctionArgument arg, PreparedStatement argUpsert, int position) throws SQLException {
+	private void addFunctionArgMutation(String functionName, FunctionArgument arg, PreparedStatement argUpsert, int position) throws SQLException {
         argUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
         argUpsert.setString(2, functionName);
         argUpsert.setString(3, arg.getArgumentType());
@@ -1043,7 +1053,8 @@ public class MetaDataClient {
                 isNull = false;
             }
             PColumn column = new PColumnImpl(PNameFactory.newName(columnName), familyName, def.getDataType(),
-                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp, false, columnQualifierBytes);
+                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp, false, columnQualifierBytes,
+                HConstants.LATEST_TIMESTAMP);
             return column;
         } catch (IllegalArgumentException e) { // Based on precondition check in constructor
             throw new SQLException(e);
@@ -1957,6 +1968,17 @@ public class MetaDataClient {
                 linkStatement.setLong(6, parent.getSequenceNumber());
                 linkStatement.setString(7, PTableType.INDEX.getSerializedValue());
                 linkStatement.execute();
+                
+                // Add row linking index table to parent table for indexes on views
+                if (parent.getType() == PTableType.VIEW) {
+	                linkStatement = connection.prepareStatement(CREATE_VIEW_INDEX_PARENT_LINK);
+	                linkStatement.setString(1, tenantIdStr);
+	                linkStatement.setString(2, schemaName);
+	                linkStatement.setString(3, tableName);
+	                linkStatement.setString(4, parent.getName().getString());
+	                linkStatement.setByte(5, LinkType.VIEW_INDEX_PARENT_TABLE.getSerializedValue());
+	                linkStatement.execute();
+                }
             }
 
             PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
@@ -2195,6 +2217,7 @@ public class MetaDataClient {
                     }
                     disableWAL = (disableWALProp == null ? parent.isWALDisabled() : disableWALProp);
                     defaultFamilyName = parent.getDefaultFamilyName() == null ? null : parent.getDefaultFamilyName().getString();
+                    // TODO PHOENIX-4766 Add an options to stop sending parent metadata when creating views
                     List<PColumn> allColumns = parent.getColumns();
                     if (saltBucketNum != null) { // Don't include salt column in columns, as it should not have it when created
                         allColumns = allColumns.subList(1, allColumns.size());
@@ -2215,6 +2238,7 @@ public class MetaDataClient {
                     linkStatement.setString(6, parent.getTenantId() == null ? null : parent.getTenantId().getString());
                     linkStatement.execute();
                     // Add row linking parent to view
+                    // TODO From 4.16 write the child links to SYSTEM.CHILD_LINK directly 
                     linkStatement = connection.prepareStatement(CREATE_CHILD_LINK);
                     linkStatement.setString(1, parent.getTenantId() == null ? null : parent.getTenantId().getString());
                     linkStatement.setString(2, parent.getSchemaName() == null ? null : parent.getSchemaName().getString());
@@ -2859,7 +2883,8 @@ public class MetaDataClient {
         String schemaName = connection.getSchema() != null && statement.getTableName().getSchemaName() == null
                 ? connection.getSchema() : statement.getTableName().getSchemaName();
         String tableName = statement.getTableName().getTableName();
-        return dropTable(schemaName, tableName, null, statement.getTableType(), statement.ifExists(), statement.cascade());
+		return dropTable(schemaName, tableName, null, statement.getTableType(), statement.ifExists(),
+				statement.cascade(), statement.getSkipAddingParentColumns());
     }
 
     public MutationState dropFunction(DropFunctionStatement statement) throws SQLException {
@@ -2870,7 +2895,7 @@ public class MetaDataClient {
         String schemaName = statement.getTableName().getSchemaName();
         String tableName = statement.getIndexName().getName();
         String parentTableName = statement.getTableName().getTableName();
-        return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
+		return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false, false);
     }
 
     private MutationState dropFunction(String functionName,
@@ -2913,8 +2938,8 @@ public class MetaDataClient {
             connection.setAutoCommit(wasAutoCommit);
         }
     }
-    private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType,
-            boolean ifExists, boolean cascade) throws SQLException {
+    MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType,
+            boolean ifExists, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -2932,7 +2957,7 @@ public class MetaDataClient {
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                 tableMetaData.add(linkDelete);
             }
-            MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade);
+            MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade, skipAddingParentColumns);
             MutationCode code = result.getMutationCode();
             PTable table = result.getTable();
             switch (code) {
@@ -3315,7 +3340,7 @@ public class MetaDataClient {
                 List<PColumn> columns = Lists.newArrayListWithExpectedSize(numCols);
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
                 Set<String> families = new LinkedHashSet<>();
-                PTable tableForCQCounters = tableType == PTableType.VIEW ? PhoenixRuntime.getTable(connection, table.getPhysicalName().getString()) : table;;
+                PTable tableForCQCounters = tableType == PTableType.VIEW ? PhoenixRuntime.getTable(connection, table.getPhysicalName().getString()) : table;
                 EncodedCQCounter cqCounterToUse = tableForCQCounters.getEncodedCQCounter();
                 Map<String, Integer> changedCqCounters = new HashMap<>(numCols);
                 if (numCols > 0 ) {
@@ -3527,9 +3552,9 @@ public class MetaDataClient {
                     if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && ! metaProperties.getNonTxToTx())) {
                         connection.addTable(result.getTable(), resolvedTimeStamp);
                         table = result.getTable();
-                    } else if (metaPropertiesEvaluated.getUpdateCacheFrequency() != null) {
-                        // Force removal from cache as the update cache frequency has changed
-                        // Note that clients outside this JVM won't be affected.
+                    } else  {
+                        // remove the table from the cache, it will be fetched from the server the
+                        // next time it is resolved
                         connection.removeTable(tenantId, fullTableName, null, resolvedTimeStamp);
                     }
                     // Delete rows in view index if we haven't dropped it already

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
index 154a9c2..a11c31a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
@@ -17,16 +17,26 @@
  */
 package org.apache.phoenix.schema;
 
-import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 
-import org.apache.phoenix.util.SchemaUtil;
+public class MetaDataSplitPolicy extends SplitOnLeadingVarCharColumnsPolicy {
 
+	@Override
+	protected boolean shouldSplit() {
+		Configuration conf = getConf();
+		return super.shouldSplit() && conf.getBoolean(QueryServices.SYSTEM_CATALOG_SPLITTABLE,
+				QueryServicesOptions.DEFAULT_SYSTEM_CATALOG_SPLITTABLE);
+	}
 
-public class MetaDataSplitPolicy extends ConstantSizeRegionSplitPolicy {
+	@Override
+	protected int getColumnToSplitAt() {
+		// SYSTEM.CATALOG rowkey is (tenant id, schema name, table name, column name,
+		// column family) ensure all meta data rows for a given schema are in the same
+		// region (indexes and tables are in the same schema as we lock the parent table
+		// when modifying an index)
+		return 2;
+	}
 
-    @Override
-    protected boolean shouldSplit() {
-        // never split SYSTEM.CATALOG
-        return false;
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
index 9e26227..2e518c4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
@@ -53,6 +53,18 @@ public interface PColumn extends PDatum {
     int getEstimatedSize();
     
     String getExpressionStr();
+
+    /**
+     * @return the cell timestamp associated with this PColumn
+     */
+    long getTimestamp();
+
+    /**
+     * @return is the column derived from some other table / view or not
+     */
+    boolean isDerived();
+
+    boolean isExcluded();
     
     /**
      * @return whether this column represents/stores the hbase cell timestamp.


[05/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 883f96d..29cf2a3 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
@@ -91,8 +91,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0 = MIN_TABLE_TIMESTAMP + 28;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0 = MIN_TABLE_TIMESTAMP + 29;
     // 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;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0;
     // Version below which we should disallow usage of mutable secondary indexing.
     public static final int MUTABLE_SI_VERSION_THRESHOLD = VersionUtil.encodeVersion("0", "94", "10");
     public static final int MAX_LOCAL_SI_VERSION_DISALLOW = VersionUtil.encodeVersion("0", "98", "8");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java
new file mode 100644
index 0000000..b1c5f65
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.coprocessor;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class TableInfo {
+
+    private final byte[] tenantId;
+    private final byte[] schema;
+    private final byte[] name;
+
+    public TableInfo(byte[] tenantId, byte[] schema, byte[] name) {
+        this.tenantId = tenantId;
+        this.schema = schema;
+        this.name = name;
+    }
+    
+    public byte[] getRowKeyPrefix() {
+        return SchemaUtil.getTableKey(tenantId, schema, name);
+    }
+
+    @Override
+    public String toString() {
+        return Bytes.toStringBinary(getRowKeyPrefix());
+    }
+    
+    public byte[] getTenantId() {
+        return tenantId;
+    }
+
+    public byte[] getSchemaName() {
+        return schema;
+    }
+
+    public byte[] getTableName() {
+        return name;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + Arrays.hashCode(name);
+        result = prime * result + Arrays.hashCode(schema);
+        result = prime * result + Arrays.hashCode(tenantId);
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        TableInfo other = (TableInfo) obj;
+        if (!Arrays.equals(name, other.name)) return false;
+        if (!Arrays.equals(schema, other.schema)) return false;
+        if (!Arrays.equals(tenantId, other.tenantId)) return false;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
new file mode 100644
index 0000000..269c5cb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.coprocessor;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+/**
+ * This class wraps the results of a scanning SYSTEM.CATALOG or SYSTEM.CHILD_LINK
+ * for child related tables or views.
+ */
+public class TableViewFinderResult {
+
+    private List<TableInfo> tableViewInfoList = Lists.newArrayList();
+
+    public TableViewFinderResult() {
+    }
+
+    public TableViewFinderResult(List<TableInfo> results) {
+        this.tableViewInfoList = results;
+    }
+
+    public boolean hasLinks() {
+        return !tableViewInfoList.isEmpty();
+    }
+
+    public List<TableInfo> getLinks() {
+        return tableViewInfoList;
+    }
+
+    void addResult(TableViewFinderResult result) {
+        this.tableViewInfoList.addAll(result.getLinks());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
new file mode 100644
index 0000000..5060d73
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
@@ -0,0 +1,144 @@
+/**
+ * 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 static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+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.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+
+public class ViewFinder {
+
+	// The PHYSICAL_TABLE link from view to the base table overwrites the PARENT_TABLE link (when namespace mapping is disabled)
+    static TableViewFinderResult findBaseTable(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
+        throws IOException {
+        return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.PHYSICAL_TABLE,
+            HConstants.LATEST_TIMESTAMP);
+    }
+    
+    static TableViewFinderResult findParentViewofIndex(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
+            throws IOException {
+            return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.VIEW_INDEX_PARENT_TABLE,
+                HConstants.LATEST_TIMESTAMP);
+        }
+
+    public static void findAllRelatives(Table systemTable, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
+        findAllRelatives(systemTable, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
+    }
+
+    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, long timestamp, TableViewFinderResult result) throws IOException {
+        TableViewFinderResult currentResult =
+            findRelatedViews(systemCatalog, tenantId, schema, table, linkType, timestamp);
+        result.addResult(currentResult);
+        for (TableInfo viewInfo : currentResult.getLinks()) {
+            findAllRelatives(systemCatalog, viewInfo.getTenantId(), viewInfo.getSchemaName(), viewInfo.getTableName(), linkType, timestamp, result);
+        }
+    }
+
+    /**
+     * Runs a scan on SYSTEM.CATALOG or SYSTEM.CHILD_LINK to get the related tables/views
+     */
+    static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, long timestamp) throws IOException {
+        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
+            throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
+        }
+        byte[] key = SchemaUtil.getTableKey(tenantId, schema, table);
+		Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        SingleColumnValueFilter linkFilter =
+            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
+                linkType.getSerializedValueAsByteArray());
+        linkFilter.setFilterIfMissing(true);
+        scan.setFilter(linkFilter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        if (linkType==PTable.LinkType.PARENT_TABLE)
+            scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+        if (linkType==PTable.LinkType.PHYSICAL_TABLE)
+            scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
+        List<TableInfo> tableInfoList = Lists.newArrayList();
+        try (ResultScanner scanner = systemCatalog.getScanner(scan))  {
+            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
+                byte[][] rowKeyMetaData = new byte[5][];
+                byte[] viewTenantId = null;
+                getVarChars(result.getRow(), 5, rowKeyMetaData);
+                if (linkType==PTable.LinkType.PARENT_TABLE) {
+                    viewTenantId = result.getValue(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+                } else if (linkType==PTable.LinkType.CHILD_TABLE) {
+                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
+                } else if (linkType==PTable.LinkType.VIEW_INDEX_PARENT_TABLE) {
+                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+                } 
+                else if (linkType==PTable.LinkType.PHYSICAL_TABLE && result.getValue(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES)!=null) {
+                    // do not links from indexes to their physical table
+                    continue;
+                }
+                byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
+                byte[] viewName = SchemaUtil.getTableNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
+                tableInfoList.add(new TableInfo(viewTenantId, viewSchemaName, viewName));
+            }
+            return new TableViewFinderResult(tableInfoList);
+        } 
+    }
+    
+    /**
+     * @return true if the given table has at least one child view
+     * @throws IOException 
+     */
+    public static boolean hasChildViews(Table systemCatalog, byte[] tenantId, byte[] schemaName, byte[] tableName, long timestamp) throws IOException {
+        byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+        Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        SingleColumnValueFilter linkFilter =
+                new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES,
+                        CompareFilter.CompareOp.EQUAL,
+                        LinkType.CHILD_TABLE.getSerializedValueAsByteArray()) {
+                    // if we found a row with the CHILD_TABLE link type we are done and can
+                    // terminate the scan
+                    @Override
+                    public boolean filterAllRemaining() throws IOException {
+                        return matchedColumn;
+                    }
+                };
+        linkFilter.setFilterIfMissing(true);
+        scan.setFilter(linkFilter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        try (ResultScanner scanner = systemCatalog.getScanner(scan)) {
+            Result result = scanner.next();
+            return result!=null; 
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
new file mode 100644
index 0000000..4528f55
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
@@ -0,0 +1,106 @@
+/**
+ * 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 static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.CreateTableCompiler;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.WhereCompiler;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.MetaDataUtil;
+
+import com.google.common.collect.Lists;
+
+
+public class WhereConstantParser {
+
+    static PTable addViewInfoToPColumnsIfNeeded(PTable view) throws SQLException {
+    	boolean[] viewColumnConstantsMatched = new boolean[view.getColumns().size()];
+        byte[][] viewColumnConstantsToBe = new byte[view.getColumns().size()][];
+        if (view.getViewStatement() == null) {
+        	return view;
+        }
+        SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
+        ParseNode whereNode = select.getWhere();
+        ColumnResolver resolver = FromCompiler.getResolver(new TableRef(view));
+        StatementContext context = new StatementContext(new PhoenixStatement(getConnectionlessConnection()), resolver);
+        Expression expression = null;
+        try {
+        	expression = WhereCompiler.compile(context, whereNode);
+        }
+        catch (ColumnNotFoundException e) {
+        	// if we could not find a column used in the view statement (which means its was dropped)
+        	// this view is not valid any more
+        	return null;
+        }
+        CreateTableCompiler.ViewWhereExpressionVisitor visitor =
+            new CreateTableCompiler.ViewWhereExpressionVisitor(view, viewColumnConstantsToBe);
+        expression.accept(visitor);
+        
+        BitSet isViewColumnReferencedToBe = new BitSet(view.getColumns().size());
+        // Used to track column references in a view
+        ExpressionCompiler expressionCompiler = new CreateTableCompiler.ColumnTrackingExpressionCompiler(context, isViewColumnReferencedToBe);
+        whereNode.accept(expressionCompiler);
+        
+        List<PColumn> result = Lists.newArrayList();
+        for (PColumn column : PTableImpl.getColumnsToClone(view)) {
+        	boolean isViewReferenced = isViewColumnReferencedToBe.get(column.getPosition());
+        	if ( (visitor.isUpdatable() || view.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(view)).equals(column)) 
+        			&& viewColumnConstantsToBe[column.getPosition()] != null) {
+				result.add(new PColumnImpl(column, viewColumnConstantsToBe[column.getPosition()], isViewReferenced));
+				viewColumnConstantsMatched[column.getPosition()]=true;
+            }
+        	// If view is not updatable, viewColumnConstants should be empty. We will still
+            // inherit our parent viewConstants, but we have no additional ones.
+        	else if(isViewReferenced ){
+        		result.add(new PColumnImpl(column, column.getViewConstant(), isViewReferenced));
+        	}
+        	else {
+                result.add(column);
+            }
+        }
+        return PTableImpl.makePTable(view, result);
+    }
+
+    private static PhoenixConnection getConnectionlessConnection() throws SQLException {
+        return DriverManager
+            .getConnection(JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + CONNECTIONLESS)
+            .unwrap(PhoenixConnection.class);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 8119c6e..744dc7e 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
@@ -105,6 +105,14 @@ public final class MetaDataProtos {
      * <code>TOO_MANY_INDEXES = 22;</code>
      */
     TOO_MANY_INDEXES(22, 22),
+    /**
+     * <code>UNABLE_TO_CREATE_CHILD_LINK = 23;</code>
+     */
+    UNABLE_TO_CREATE_CHILD_LINK(23, 23),
+    /**
+     * <code>UNABLE_TO_UPDATE_PARENT_TABLE = 24;</code>
+     */
+    UNABLE_TO_UPDATE_PARENT_TABLE(24, 24),
     ;
 
     /**
@@ -199,6 +207,14 @@ public final class MetaDataProtos {
      * <code>TOO_MANY_INDEXES = 22;</code>
      */
     public static final int TOO_MANY_INDEXES_VALUE = 22;
+    /**
+     * <code>UNABLE_TO_CREATE_CHILD_LINK = 23;</code>
+     */
+    public static final int UNABLE_TO_CREATE_CHILD_LINK_VALUE = 23;
+    /**
+     * <code>UNABLE_TO_UPDATE_PARENT_TABLE = 24;</code>
+     */
+    public static final int UNABLE_TO_UPDATE_PARENT_TABLE_VALUE = 24;
 
 
     public final int getNumber() { return value; }
@@ -228,6 +244,8 @@ public final class MetaDataProtos {
         case 20: return AUTO_PARTITION_SEQUENCE_NOT_FOUND;
         case 21: return CANNOT_COERCE_AUTO_PARTITION_ID;
         case 22: return TOO_MANY_INDEXES;
+        case 23: return UNABLE_TO_CREATE_CHILD_LINK;
+        case 24: return UNABLE_TO_UPDATE_PARENT_TABLE;
         default: return null;
       }
     }
@@ -3948,6 +3966,40 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 6;</code>
      */
     int getClientVersion();
+
+    // optional bool skipAddingParentColumns = 7;
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    boolean hasSkipAddingParentColumns();
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    boolean getSkipAddingParentColumns();
+
+    // optional bool skipAddingIndexes = 8;
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    boolean hasSkipAddingIndexes();
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    boolean getSkipAddingIndexes();
+
+    // optional .PTable lockedAncestorTable = 9;
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    boolean hasLockedAncestorTable();
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable();
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder();
   }
   /**
    * Protobuf type {@code GetTableRequest}
@@ -4030,6 +4082,29 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              skipAddingParentColumns_ = input.readBool();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000080;
+              skipAddingIndexes_ = input.readBool();
+              break;
+            }
+            case 74: {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000100) == 0x00000100)) {
+                subBuilder = lockedAncestorTable_.toBuilder();
+              }
+              lockedAncestorTable_ = input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(lockedAncestorTable_);
+                lockedAncestorTable_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000100;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4166,6 +4241,60 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional bool skipAddingParentColumns = 7;
+    public static final int SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER = 7;
+    private boolean skipAddingParentColumns_;
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    public boolean hasSkipAddingParentColumns() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    public boolean getSkipAddingParentColumns() {
+      return skipAddingParentColumns_;
+    }
+
+    // optional bool skipAddingIndexes = 8;
+    public static final int SKIPADDINGINDEXES_FIELD_NUMBER = 8;
+    private boolean skipAddingIndexes_;
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    public boolean hasSkipAddingIndexes() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    public boolean getSkipAddingIndexes() {
+      return skipAddingIndexes_;
+    }
+
+    // optional .PTable lockedAncestorTable = 9;
+    public static final int LOCKEDANCESTORTABLE_FIELD_NUMBER = 9;
+    private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable lockedAncestorTable_;
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    public boolean hasLockedAncestorTable() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable() {
+      return lockedAncestorTable_;
+    }
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder() {
+      return lockedAncestorTable_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       schemaName_ = com.google.protobuf.ByteString.EMPTY;
@@ -4173,6 +4302,9 @@ public final class MetaDataProtos {
       tableTimestamp_ = 0L;
       clientTimestamp_ = 0L;
       clientVersion_ = 0;
+      skipAddingParentColumns_ = false;
+      skipAddingIndexes_ = false;
+      lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4199,6 +4331,12 @@ public final class MetaDataProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (hasLockedAncestorTable()) {
+        if (!getLockedAncestorTable().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -4224,6 +4362,15 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeInt32(6, clientVersion_);
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeBool(7, skipAddingParentColumns_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBool(8, skipAddingIndexes_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeMessage(9, lockedAncestorTable_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4257,6 +4404,18 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(6, clientVersion_);
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(7, skipAddingParentColumns_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(8, skipAddingIndexes_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(9, lockedAncestorTable_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4310,6 +4469,21 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasSkipAddingParentColumns() == other.hasSkipAddingParentColumns());
+      if (hasSkipAddingParentColumns()) {
+        result = result && (getSkipAddingParentColumns()
+            == other.getSkipAddingParentColumns());
+      }
+      result = result && (hasSkipAddingIndexes() == other.hasSkipAddingIndexes());
+      if (hasSkipAddingIndexes()) {
+        result = result && (getSkipAddingIndexes()
+            == other.getSkipAddingIndexes());
+      }
+      result = result && (hasLockedAncestorTable() == other.hasLockedAncestorTable());
+      if (hasLockedAncestorTable()) {
+        result = result && getLockedAncestorTable()
+            .equals(other.getLockedAncestorTable());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4347,6 +4521,18 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasSkipAddingParentColumns()) {
+        hash = (37 * hash) + SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipAddingParentColumns());
+      }
+      if (hasSkipAddingIndexes()) {
+        hash = (37 * hash) + SKIPADDINGINDEXES_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipAddingIndexes());
+      }
+      if (hasLockedAncestorTable()) {
+        hash = (37 * hash) + LOCKEDANCESTORTABLE_FIELD_NUMBER;
+        hash = (53 * hash) + getLockedAncestorTable().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4448,6 +4634,7 @@ public final class MetaDataProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getLockedAncestorTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -4468,6 +4655,16 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000020);
+        skipAddingParentColumns_ = false;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        skipAddingIndexes_ = false;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+        } else {
+          lockedAncestorTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000100);
         return this;
       }
 
@@ -4520,6 +4717,22 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000020;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.skipAddingParentColumns_ = skipAddingParentColumns_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.skipAddingIndexes_ = skipAddingIndexes_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        if (lockedAncestorTableBuilder_ == null) {
+          result.lockedAncestorTable_ = lockedAncestorTable_;
+        } else {
+          result.lockedAncestorTable_ = lockedAncestorTableBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4554,6 +4767,15 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasSkipAddingParentColumns()) {
+          setSkipAddingParentColumns(other.getSkipAddingParentColumns());
+        }
+        if (other.hasSkipAddingIndexes()) {
+          setSkipAddingIndexes(other.getSkipAddingIndexes());
+        }
+        if (other.hasLockedAncestorTable()) {
+          mergeLockedAncestorTable(other.getLockedAncestorTable());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4579,6 +4801,12 @@ public final class MetaDataProtos {
           
           return false;
         }
+        if (hasLockedAncestorTable()) {
+          if (!getLockedAncestorTable().isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -4808,6 +5036,189 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional bool skipAddingParentColumns = 7;
+      private boolean skipAddingParentColumns_ ;
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public boolean hasSkipAddingParentColumns() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public boolean getSkipAddingParentColumns() {
+        return skipAddingParentColumns_;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public Builder setSkipAddingParentColumns(boolean value) {
+        bitField0_ |= 0x00000040;
+        skipAddingParentColumns_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public Builder clearSkipAddingParentColumns() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        skipAddingParentColumns_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool skipAddingIndexes = 8;
+      private boolean skipAddingIndexes_ ;
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public boolean hasSkipAddingIndexes() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public boolean getSkipAddingIndexes() {
+        return skipAddingIndexes_;
+      }
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public Builder setSkipAddingIndexes(boolean value) {
+        bitField0_ |= 0x00000080;
+        skipAddingIndexes_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public Builder clearSkipAddingIndexes() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        skipAddingIndexes_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional .PTable lockedAncestorTable = 9;
+      private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> lockedAncestorTableBuilder_;
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public boolean hasLockedAncestorTable() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable() {
+        if (lockedAncestorTableBuilder_ == null) {
+          return lockedAncestorTable_;
+        } else {
+          return lockedAncestorTableBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder setLockedAncestorTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (lockedAncestorTableBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          lockedAncestorTable_ = value;
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder setLockedAncestorTable(
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder builderForValue) {
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTable_ = builderForValue.build();
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder mergeLockedAncestorTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (lockedAncestorTableBuilder_ == null) {
+          if (((bitField0_ & 0x00000100) == 0x00000100) &&
+              lockedAncestorTable_ != org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance()) {
+            lockedAncestorTable_ =
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.newBuilder(lockedAncestorTable_).mergeFrom(value).buildPartial();
+          } else {
+            lockedAncestorTable_ = value;
+          }
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder clearLockedAncestorTable() {
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000100);
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder getLockedAncestorTableBuilder() {
+        bitField0_ |= 0x00000100;
+        onChanged();
+        return getLockedAncestorTableFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder() {
+        if (lockedAncestorTableBuilder_ != null) {
+          return lockedAncestorTableBuilder_.getMessageOrBuilder();
+        } else {
+          return lockedAncestorTable_;
+        }
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> 
+          getLockedAncestorTableFieldBuilder() {
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder>(
+                  lockedAncestorTable_,
+                  getParentForChildren(),
+                  isClean());
+          lockedAncestorTable_ = null;
+        }
+        return lockedAncestorTableBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:GetTableRequest)
     }
 
@@ -8780,6 +9191,16 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 4;</code>
      */
     int getClientVersion();
+
+    // optional bool skipAddingParentColumns = 5;
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    boolean hasSkipAddingParentColumns();
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    boolean getSkipAddingParentColumns();
   }
   /**
    * Protobuf type {@code DropTableRequest}
@@ -8855,6 +9276,11 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              skipAddingParentColumns_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -8996,11 +9422,28 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional bool skipAddingParentColumns = 5;
+    public static final int SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER = 5;
+    private boolean skipAddingParentColumns_;
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    public boolean hasSkipAddingParentColumns() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    public boolean getSkipAddingParentColumns() {
+      return skipAddingParentColumns_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       tableType_ = "";
       cascade_ = false;
       clientVersion_ = 0;
+      skipAddingParentColumns_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -9030,6 +9473,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt32(4, clientVersion_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(5, skipAddingParentColumns_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -9060,6 +9506,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(4, clientVersion_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, skipAddingParentColumns_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -9100,6 +9550,11 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasSkipAddingParentColumns() == other.hasSkipAddingParentColumns());
+      if (hasSkipAddingParentColumns()) {
+        result = result && (getSkipAddingParentColumns()
+            == other.getSkipAddingParentColumns());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -9129,6 +9584,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasSkipAddingParentColumns()) {
+        hash = (37 * hash) + SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipAddingParentColumns());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -9246,6 +9705,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000004);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
+        skipAddingParentColumns_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -9291,6 +9752,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000004;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.skipAddingParentColumns_ = skipAddingParentColumns_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -9328,6 +9793,9 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasSkipAddingParentColumns()) {
+          setSkipAddingParentColumns(other.getSkipAddingParentColumns());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9571,6 +10039,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional bool skipAddingParentColumns = 5;
+      private boolean skipAddingParentColumns_ ;
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public boolean hasSkipAddingParentColumns() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public boolean getSkipAddingParentColumns() {
+        return skipAddingParentColumns_;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public Builder setSkipAddingParentColumns(boolean value) {
+        bitField0_ |= 0x00000010;
+        skipAddingParentColumns_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public Builder clearSkipAddingParentColumns() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        skipAddingParentColumns_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:DropTableRequest)
     }
 
@@ -17158,83 +17659,88 @@ public final class MetaDataProtos {
       "\030\007 \001(\014\022\034\n\010function\030\010 \003(\0132\n.PFunction\022/\n\024" +
       "sharedTablesToDelete\030\t \003(\0132\021.SharedTable" +
       "State\022\030\n\006schema\030\n \001(\0132\010.PSchema\022\030\n\020autoP" +
-      "artitionNum\030\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\005\"\222" +
+      "artitionNum\030\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\005\"\364" +
       "\001\n\017GetTableRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\n" +
       "schemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\026\n\016t" +
       "ableTimestamp\030\004 \002(\003\022\027\n\017clientTimestamp\030\005" +
-      " \002(\003\022\025\n\rclientVersion\030\006 \001(\005\"\212\001\n\023GetFunct" +
-      "ionsRequest\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctio" +
-      "nNames\030\002 \003(\014\022\032\n\022functionTimestamps\030\003 \003(\003",
-      "\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rclientVersi" +
-      "on\030\005 \001(\005\"V\n\020GetSchemaRequest\022\022\n\nschemaNa" +
-      "me\030\001 \002(\t\022\027\n\017clientTimestamp\030\002 \002(\003\022\025\n\rcli" +
-      "entVersion\030\003 \002(\005\"d\n\022CreateTableRequest\022\036" +
-      "\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclient" +
-      "Version\030\002 \001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r" +
-      "\n\025CreateFunctionRequest\022\036\n\026tableMetadata" +
-      "Mutations\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007re" +
-      "place\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023Cr" +
-      "eateSchemaRequest\022\036\n\026tableMetadataMutati",
-      "ons\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientV" +
-      "ersion\030\003 \002(\005\"m\n\020DropTableRequest\022\036\n\026tabl" +
-      "eMetadataMutations\030\001 \003(\014\022\021\n\ttableType\030\002 " +
-      "\002(\t\022\017\n\007cascade\030\003 \001(\010\022\025\n\rclientVersion\030\004 " +
-      "\001(\005\"_\n\021DropSchemaRequest\022\037\n\027schemaMetada" +
-      "taMutations\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n" +
-      "\rclientVersion\030\003 \002(\005\"I\n\020AddColumnRequest" +
-      "\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclie" +
-      "ntVersion\030\002 \001(\005\"J\n\021DropColumnRequest\022\036\n\026" +
-      "tableMetadataMutations\030\001 \003(\014\022\025\n\rclientVe",
-      "rsion\030\002 \001(\005\"^\n\023DropFunctionRequest\022\036\n\026ta" +
-      "bleMetadataMutations\030\001 \003(\014\022\020\n\010ifExists\030\002" +
-      " \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"P\n\027UpdateInd" +
-      "exStateRequest\022\036\n\026tableMetadataMutations" +
-      "\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"*\n\021ClearCa" +
-      "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\"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",
+      " \002(\003\022\025\n\rclientVersion\030\006 \001(\005\022\037\n\027skipAddin" +
+      "gParentColumns\030\007 \001(\010\022\031\n\021skipAddingIndexe" +
+      "s\030\010 \001(\010\022$\n\023lockedAncestorTable\030\t \001(\0132\007.P",
+      "Table\"\212\001\n\023GetFunctionsRequest\022\020\n\010tenantI" +
+      "d\030\001 \002(\014\022\025\n\rfunctionNames\030\002 \003(\014\022\032\n\022functi" +
+      "onTimestamps\030\003 \003(\003\022\027\n\017clientTimestamp\030\004 " +
+      "\002(\003\022\025\n\rclientVersion\030\005 \001(\005\"V\n\020GetSchemaR" +
+      "equest\022\022\n\nschemaName\030\001 \002(\t\022\027\n\017clientTime" +
+      "stamp\030\002 \002(\003\022\025\n\rclientVersion\030\003 \002(\005\"d\n\022Cr" +
+      "eateTableRequest\022\036\n\026tableMetadataMutatio" +
+      "ns\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\022\027\n\017alloc" +
+      "ateIndexId\030\003 \001(\010\"r\n\025CreateFunctionReques" +
+      "t\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\021\n\ttem",
+      "porary\030\002 \002(\010\022\017\n\007replace\030\003 \001(\010\022\025\n\rclientV" +
+      "ersion\030\004 \001(\005\"`\n\023CreateSchemaRequest\022\036\n\026t" +
+      "ableMetadataMutations\030\001 \003(\014\022\022\n\nschemaNam" +
+      "e\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"\216\001\n\020DropT" +
+      "ableRequest\022\036\n\026tableMetadataMutations\030\001 " +
+      "\003(\014\022\021\n\ttableType\030\002 \002(\t\022\017\n\007cascade\030\003 \001(\010\022" +
+      "\025\n\rclientVersion\030\004 \001(\005\022\037\n\027skipAddingPare" +
+      "ntColumns\030\005 \001(\010\"_\n\021DropSchemaRequest\022\037\n\027" +
+      "schemaMetadataMutations\030\001 \003(\014\022\022\n\nschemaN" +
+      "ame\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"I\n\020AddC",
+      "olumnRequest\022\036\n\026tableMetadataMutations\030\001" +
+      " \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"J\n\021DropColum" +
+      "nRequest\022\036\n\026tableMetadataMutations\030\001 \003(\014" +
+      "\022\025\n\rclientVersion\030\002 \001(\005\"^\n\023DropFunctionR" +
+      "equest\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\020" +
+      "\n\010ifExists\030\002 \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"" +
+      "P\n\027UpdateIndexStateRequest\022\036\n\026tableMetad" +
+      "ataMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(" +
+      "\005\"*\n\021ClearCacheRequest\022\025\n\rclientVersion\030" +
+      "\001 \001(\005\"*\n\022ClearCacheResponse\022\024\n\014unfreedBy",
+      "tes\030\001 \001(\003\"*\n\021GetVersionRequest\022\025\n\rclient" +
+      "Version\030\001 \001(\005\"E\n\022GetVersionResponse\022\017\n\007v" +
+      "ersion\030\001 \002(\003\022\036\n\026systemCatalogTimestamp\030\002" +
+      " \001(\003\"\205\001\n\032ClearTableFromCacheRequest\022\020\n\010t" +
+      "enantId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttab" +
+      "leName\030\003 \002(\014\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n" +
+      "\rclientVersion\030\005 \001(\005\"\035\n\033ClearTableFromCa" +
+      "cheResponse*\271\005\n\014MutationCode\022\030\n\024TABLE_AL" +
+      "READY_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020C" +
+      "OLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXIS",
+      "TS\020\003\022\035\n\031CONCURRENT_TABLE_MUTATION\020\004\022\027\n\023T" +
+      "ABLE_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABLE_FOUN" +
+      "D\020\006\022\034\n\030UNALLOWED_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\022FUNCTIO" +
+      "N_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_AL" +
+      "READY_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_RE" +
+      "GION\020\021\022\032\n\026TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UN",
+      "ALLOWED_SCHEMA_MUTATION\020\023\022%\n!AUTO_PARTIT" +
+      "ION_SEQUENCE_NOT_FOUND\020\024\022#\n\037CANNOT_COERC" +
+      "E_AUTO_PARTITION_ID\020\025\022\024\n\020TOO_MANY_INDEXE" +
+      "S\020\026\022\037\n\033UNABLE_TO_CREATE_CHILD_LINK\020\027\022!\n\035" +
+      "UNABLE_TO_UPDATE_PARENT_TABLE\020\0302\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" +
+      "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",
+      "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" +
+      "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",
+      "\032\023.GetVersionResponse\022P\n\023clearTableFromC" +
       "ache\022\033.ClearTableFromCacheRequest\032\034.Clea" +
       "rTableFromCacheResponseBB\n(org.apache.ph" +
       "oenix.coprocessor.generatedB\016MetaDataPro" +
@@ -17262,7 +17768,7 @@ public final class MetaDataProtos {
           internal_static_GetTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetTableRequest_descriptor,
-              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", "SkipAddingParentColumns", "SkipAddingIndexes", "LockedAncestorTable", });
           internal_static_GetFunctionsRequest_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_GetFunctionsRequest_fieldAccessorTable = new
@@ -17298,7 +17804,7 @@ public final class MetaDataProtos {
           internal_static_DropTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", });
+              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", "SkipAddingParentColumns", });
           internal_static_DropSchemaRequest_descriptor =
             getDescriptor().getMessageTypes().get(9);
           internal_static_DropSchemaRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 8d500e8..ab61826 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
@@ -140,17 +140,17 @@ public final class PTableProtos {
      */
     com.google.protobuf.ByteString getFamilyNameBytes();
 
-    // required string dataType = 3;
+    // optional string dataType = 3;
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     boolean hasDataType();
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     java.lang.String getDataType();
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     com.google.protobuf.ByteString
         getDataTypeBytes();
@@ -195,13 +195,13 @@ public final class PTableProtos {
      */
     int getPosition();
 
-    // required int32 sortOrder = 8;
+    // optional int32 sortOrder = 8;
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     boolean hasSortOrder();
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     int getSortOrder();
 
@@ -279,6 +279,26 @@ public final class PTableProtos {
      * <code>optional bytes columnQualifierBytes = 15;</code>
      */
     com.google.protobuf.ByteString getColumnQualifierBytes();
+
+    // optional int64 timestamp = 16;
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    boolean hasTimestamp();
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    long getTimestamp();
+
+    // optional bool derived = 17 [default = false];
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    boolean hasDerived();
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    boolean getDerived();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -406,6 +426,16 @@ public final class PTableProtos {
               columnQualifierBytes_ = input.readBytes();
               break;
             }
+            case 128: {
+              bitField0_ |= 0x00008000;
+              timestamp_ = input.readInt64();
+              break;
+            }
+            case 136: {
+              bitField0_ |= 0x00010000;
+              derived_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -478,17 +508,17 @@ public final class PTableProtos {
       return familyNameBytes_;
     }
 
-    // required string dataType = 3;
+    // optional string dataType = 3;
     public static final int DATATYPE_FIELD_NUMBER = 3;
     private java.lang.Object dataType_;
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     public boolean hasDataType() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     public java.lang.String getDataType() {
       java.lang.Object ref = dataType_;
@@ -505,7 +535,7 @@ public final class PTableProtos {
       }
     }
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     public com.google.protobuf.ByteString
         getDataTypeBytes() {
@@ -585,17 +615,17 @@ public final class PTableProtos {
       return position_;
     }
 
-    // required int32 sortOrder = 8;
+    // optional int32 sortOrder = 8;
     public static final int SORTORDER_FIELD_NUMBER = 8;
     private int sortOrder_;
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     public boolean hasSortOrder() {
       return ((bitField0_ & 0x00000080) == 0x00000080);
     }
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     public int getSortOrder() {
       return sortOrder_;
@@ -740,6 +770,38 @@ public final class PTableProtos {
       return columnQualifierBytes_;
     }
 
+    // optional int64 timestamp = 16;
+    public static final int TIMESTAMP_FIELD_NUMBER = 16;
+    private long timestamp_;
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    public boolean hasTimestamp() {
+      return ((bitField0_ & 0x00008000) == 0x00008000);
+    }
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    public long getTimestamp() {
+      return timestamp_;
+    }
+
+    // optional bool derived = 17 [default = false];
+    public static final int DERIVED_FIELD_NUMBER = 17;
+    private boolean derived_;
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    public boolean hasDerived() {
+      return ((bitField0_ & 0x00010000) == 0x00010000);
+    }
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    public boolean getDerived() {
+      return derived_;
+    }
+
     private void initFields() {
       columnNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       familyNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -756,6 +818,8 @@ public final class PTableProtos {
       isRowTimestamp_ = false;
       isDynamic_ = false;
       columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
+      timestamp_ = 0L;
+      derived_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -766,10 +830,6 @@ public final class PTableProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasDataType()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       if (!hasNullable()) {
         memoizedIsInitialized = 0;
         return false;
@@ -778,10 +838,6 @@ public final class PTableProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasSortOrder()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -834,6 +890,12 @@ public final class PTableProtos {
       if (((bitField0_ & 0x00004000) == 0x00004000)) {
         output.writeBytes(15, columnQualifierBytes_);
       }
+      if (((bitField0_ & 0x00008000) == 0x00008000)) {
+        output.writeInt64(16, timestamp_);
+      }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        output.writeBool(17, derived_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -903,6 +965,14 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(15, columnQualifierBytes_);
       }
+      if (((bitField0_ & 0x00008000) == 0x00008000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(16, timestamp_);
+      }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(17, derived_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -1001,6 +1071,16 @@ public final class PTableProtos {
         result = result && getColumnQualifierBytes()
             .equals(other.getColumnQualifierBytes());
       }
+      result = result && (hasTimestamp() == other.hasTimestamp());
+      if (hasTimestamp()) {
+        result = result && (getTimestamp()
+            == other.getTimestamp());
+      }
+      result = result && (hasDerived() == other.hasDerived());
+      if (hasDerived()) {
+        result = result && (getDerived()
+            == other.getDerived());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1074,6 +1154,14 @@ public final class PTableProtos {
         hash = (37 * hash) + COLUMNQUALIFIERBYTES_FIELD_NUMBER;
         hash = (53 * hash) + getColumnQualifierBytes().hashCode();
       }
+      if (hasTimestamp()) {
+        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimestamp());
+      }
+      if (hasDerived()) {
+        hash = (37 * hash) + DERIVED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getDerived());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1213,6 +1301,10 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00002000);
         columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00004000);
+        timestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00008000);
+        derived_ = false;
+        bitField0_ = (bitField0_ & ~0x00010000);
         return this;
       }
 
@@ -1301,6 +1393,14 @@ public final class PTableProtos {
           to_bitField0_ |= 0x00004000;
         }
         result.columnQualifierBytes_ = columnQualifierBytes_;
+        if (((from_bitField0_ & 0x00008000) == 0x00008000)) {
+          to_bitField0_ |= 0x00008000;
+        }
+        result.timestamp_ = timestamp_;
+        if (((from_bitField0_ & 0x00010000) == 0x00010000)) {
+          to_bitField0_ |= 0x00010000;
+        }
+        result.derived_ = derived_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1366,6 +1466,12 @@ public final class PTableProtos {
         if (other.hasColumnQualifierBytes()) {
           setColumnQualifierBytes(other.getColumnQualifierBytes());
         }
+        if (other.hasTimestamp()) {
+          setTimestamp(other.getTimestamp());
+        }
+        if (other.hasDerived()) {
+          setDerived(other.getDerived());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1375,10 +1481,6 @@ public final class PTableProtos {
           
           return false;
         }
-        if (!hasDataType()) {
-          
-          return false;
-        }
         if (!hasNullable()) {
           
           return false;
@@ -1387,10 +1489,6 @@ public final class PTableProtos {
           
           return false;
         }
-        if (!hasSortOrder()) {
-          
-          return false;
-        }
         return true;
       }
 
@@ -1485,16 +1583,16 @@ public final class PTableProtos {
         return this;
       }
 
-      // required string dataType = 3;
+      // optional string dataType = 3;
       private java.lang.Object dataType_ = "";
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public boolean hasDataType() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public java.lang.String getDataType() {
         java.lang.Object ref = dataType_;
@@ -1508,7 +1606,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public com.google.protobuf.ByteString
           getDataTypeBytes() {
@@ -1524,7 +1622,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public Builder setDataType(
           java.lang.String value) {
@@ -1537,7 +1635,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public Builder clearDataType() {
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -1546,7 +1644,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public Builder setDataTypeBytes(
           com.google.protobuf.ByteString value) {
@@ -1691,22 +1789,22 @@ public final class PTableProtos {
         return this;
       }
 
-      // required int32 sortOrder = 8;
+      // optional int32 sortOrder = 8;
       private int sortOrder_ ;
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public boolean hasSortOrder() {
         return ((bitField0_ & 0x00000080) == 0x00000080);
       }
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public int getSortOrder() {
         return sortOrder_;
       }
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public Builder setSortOrder(int value) {
         bitField0_ |= 0x00000080;
@@ -1715,7 +1813,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public Builder clearSortOrder() {
         bitField0_ = (bitField0_ & ~0x00000080);
@@ -2002,6 +2100,72 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int64 timestamp = 16;
+      private long timestamp_ ;
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public boolean hasTimestamp() {
+        return ((bitField0_ & 0x00008000) == 0x00008000);
+      }
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public long getTimestamp() {
+        return timestamp_;
+      }
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public Builder setTimestamp(long value) {
+        bitField0_ |= 0x00008000;
+        timestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public Builder clearTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00008000);
+        timestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional bool derived = 17 [default = false];
+      private boolean derived_ ;
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public boolean hasDerived() {
+        return ((bitField0_ & 0x00010000) == 0x00010000);
+      }
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public boolean getDerived() {
+        return derived_;
+      }
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public Builder setDerived(boolean value) {
+        bitField0_ |= 0x00010000;
+        derived_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public Builder clearDerived() {
+        bitField0_ = (bitField0_ & ~0x00010000);
+        derived_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PColumn)
     }
 
@@ -8668,48 +8832,49 @@ public final class PTableProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\014PTable.proto\032\021PGuidePosts.proto\"\304\002\n\007PC" +
+      "\n\014PTable.proto\032\021PGuidePosts.proto\"\357\002\n\007PC" +
       "olumn\022\027\n\017columnNameBytes\030\001 \002(\014\022\027\n\017family" +
-      "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \002(\t\022\021\n\tmax" +
+      "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \001(\t\022\021\n\tmax" +
       "Length\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010nullable\030" +
-      "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \002" +
+      "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \001" +
       "(\005\022\021\n\tarraySize\030\t \001(\005\022\024\n\014viewConstant\030\n " +
       "\001(\014\022\026\n\016viewReferenced\030\013 \001(\010\022\022\n\nexpressio" +
       "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\022\021\n\tisDyn" +
       "amic\030\016 \001(\010\022\034\n\024columnQualifierBytes\030\017 \001(\014" +
-      "\"\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\"\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" +
-      "eNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pkNa" +
-      "meBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007colu" +
-      "mns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007.P" +
-      "Table\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022dataTa",
-      "bleNameBytes\030\r \001(\014\022\031\n\021defaultFamilyName\030" +
-      "\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTenant" +
-      "\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewStatemen" +
-      "t\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010tenant" +
-      "Id\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tindexTy" +
-      "pe\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\nstor" +
-      "eNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005\022\036\n" +
-      "\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtransac" +
-      "tional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030\034 \001" +
-      "(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021isN",
-      "amespaceMapped\030\036 \001(\010\022\034\n\024autoParititonSeq" +
-      "Name\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(\010\022\027" +
-      "\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\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"
+      "\022\021\n\ttimestamp\030\020 \001(\003\022\026\n\007derived\030\021 \001(\010:\005fa",
+      "lse\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006valu" +
+      "es\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n" +
+      "\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030" +
+      "\005 \001(\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\016" +
+      "tableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162" +
+      "\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequ" +
+      "enceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013p" +
+      "kNameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007c" +
+      "olumns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132",
+      "\007.PTable\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022dat" +
+      "aTableNameBytes\030\r \001(\014\022\031\n\021defaultFamilyNa" +
+      "me\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTen" +
+      "ant\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewState" +
+      "ment\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010ten" +
+      "antId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tinde" +
+      "xType\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\ns" +
+      "toreNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005" +
+      "\022\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtran" +
+      "sactional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030",
+      "\034 \001(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021" +
+      "isNamespaceMapped\030\036 \001(\010\022\034\n\024autoParititon" +
+      "SeqName\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(" +
+      "\010\022\027\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageSch" +
+      "eme\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021enc" +
+      "odedCQCounters\030$ \003(\0132\021.EncodedCQCounter\022" +
+      "\"\n\032useStatsForParallelization\030% \001(\010\022\033\n\023t" +
+      "ransactionProvider\030& \001(\005\"6\n\020EncodedCQCou" +
+      "nter\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\004",
+      "VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apach" +
+      "e.phoenix.coprocessor.generatedB\014PTableP" +
+      "rotosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8721,7 +8886,7 @@ public final class PTableProtos {
           internal_static_PColumn_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PColumn_descriptor,
-              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "ColumnQualifierBytes", });
+              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "ColumnQualifierBytes", "Timestamp", "Derived", });
           internal_static_PTableStats_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_PTableStats_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
index ec4aa3a..532012f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
@@ -147,7 +147,7 @@ public abstract class LikeExpression extends BaseCompoundExpression {
         }
     }
 
-    private static String toPattern(String s) {
+    public static String toPattern(String s) {
         StringBuilder sb = new StringBuilder(s.length());
 
         // From the JDK doc: \Q and \E protect everything between them

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 312d17b..3d9b261 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
@@ -209,13 +209,11 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
 
     public PhoenixConnection(PhoenixConnection connection, long scn)
             throws SQLException {
-        this(connection.getQueryServices(), connection, scn);
+        this(connection, newPropsWithSCN(scn, connection.getClientInfo()));
     }
 
-    public PhoenixConnection(ConnectionQueryServices services,
-            PhoenixConnection connection, long scn) throws SQLException {
-        this(services, connection.getURL(), newPropsWithSCN(scn,
-                connection.getClientInfo()), connection.metaData, connection
+	public PhoenixConnection(PhoenixConnection connection, Properties props) throws SQLException {
+        this(connection.getQueryServices(), connection.getURL(), props, connection.metaData, connection
                 .getMutationState(), connection.isDescVarLengthRowKeyUpgrade(),
                 connection.isRunningUpgrade(), connection.buildingIndex);
         this.isAutoCommit = connection.isAutoCommit;


[06/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 ae2fa66..5e8a5dc 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
@@ -17,8 +17,6 @@
  */
 package org.apache.phoenix.coprocessor;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
 import static org.apache.hadoop.hbase.KeyValueUtil.createFirstOnRow;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE_BYTES;
@@ -55,7 +53,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT_BYTES
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_ARGS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES;
@@ -78,9 +75,8 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_BYTE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE_BYTES;
 import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
 import static org.apache.phoenix.schema.PTableType.INDEX;
-import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
+import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
@@ -91,14 +87,16 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NavigableMap;
+import java.util.Properties;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
@@ -108,26 +106,21 @@ 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.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
-import org.apache.hadoop.hbase.filter.PageFilter;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
 import org.apache.hadoop.hbase.ipc.RpcUtil;
@@ -140,6 +133,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.GlobalCache.FunctionBytesPtr;
+import org.apache.phoenix.compile.ColumnNameTrackingExpressionCompiler;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.QueryPlan;
@@ -183,6 +177,7 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.metrics.Metrics;
+import org.apache.phoenix.parse.DropTableStatement;
 import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PFunction.FunctionArgument;
@@ -190,6 +185,7 @@ import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.protobuf.ProtobufUtil;
+import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
@@ -197,6 +193,8 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PColumnImpl;
@@ -212,16 +210,17 @@ import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.ParentTableNotFoundException;
+import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceAlreadyExistsException;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SequenceNotFoundException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
-import org.apache.phoenix.schema.TableProperty;
 import org.apache.phoenix.schema.TableRef;
-import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
@@ -239,6 +238,7 @@ import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.IndexUtil;
 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;
@@ -248,6 +248,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.cache.Cache;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.protobuf.ByteString;
@@ -256,17 +257,35 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 
 /**
- *
  * Endpoint co-processor through which all Phoenix metadata mutations flow.
+ * Phoenix metadata is stored in SYSTEM.CATALOG. The table specific information
+ * is stored in a single header row. Column information is stored in a separate
+ * row per column. Linking information (indexes, views etc) are stored using a
+ * separate row for each link that uses the {@link LinkType} column value. The
+ * parent->child links are stored in a separate SYSTEM.CHILD_LINK table.
+ * Metadata for all tables/views/indexes in the same schema are stored in a
+ * single region which is enforced using the {@link MetaDataSplitPolicy}.
+ * 
+ * While creating child views we only store columns added by the view. When
+ * resolving a view we resolve all its parents and add their columns to the
+ * PTable that is returned. We lock the parent table while creating an index to
+ * ensure its metadata doesn't change.
+ * While adding or dropping columns we lock the table or view to ensure that
+ * concurrent conflicting changes are prevented. We also validate that there are
+ * no existing conflicting child view columns when we add a column to a parent.
+ * While dropping a column from a parent we check if there are any child views
+ * that need the column and throw an exception. If there are view indexes that
+ * required the column we drop them as well.
+ * While dropping a table or view that has children using the cascade option, we
+ * do not drop the child view metadata which will be removed at compaction time.
+ * If we recreate a table or view that was dropped whose child metadata hasn't
+ * been removed yet, we delete the child view metadata. When resolving a view,
+ * we resolve all its parents, if any of them are dropped the child view is
+ * considered to be dropped and we throw a TableNotFoundException.
+ * 
  * We only allow mutations to the latest version of a Phoenix table (i.e. the
- * timeStamp must be increasing).
- * For adding/dropping columns use a sequence number on the table to ensure that
- * the client has the latest version.
- * The timeStamp on the table correlates with the timeStamp on the data row.
- * TODO: we should enforce that a metadata mutation uses a timeStamp bigger than
- * any in use on the data table, b/c otherwise we can end up with data rows that
- * are not valid against a schema row.
- *
+ * timeStamp must be increasing). For adding/dropping columns we use a sequence
+ * number on the table to ensure that the client has the latest version.
  *
  * @since 0.1
  */
@@ -423,7 +442,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int COLUMN_DEF_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_DEF_KV);
     private static final int IS_ROW_TIMESTAMP_INDEX = COLUMN_KV_COLUMNS.indexOf(IS_ROW_TIMESTAMP_KV);
     private static final int COLUMN_QUALIFIER_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_QUALIFIER_KV);
-    
+
     private static final int LINK_TYPE_INDEX = 0;
 
     private static final KeyValue CLASS_NAME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, CLASS_NAME_BYTES);
@@ -447,7 +466,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     static {
         Collections.sort(FUNCTION_KV_COLUMNS, KeyValue.COMPARATOR);
     }
-    
+
     private static final int CLASS_NAME_INDEX = FUNCTION_KV_COLUMNS.indexOf(CLASS_NAME_KV);
     private static final int JAR_PATH_INDEX = FUNCTION_KV_COLUMNS.indexOf(JAR_PATH_KV);
     private static final int RETURN_TYPE_INDEX = FUNCTION_KV_COLUMNS.indexOf(RETURN_TYPE_KV);
@@ -464,7 +483,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     static {
         Collections.sort(FUNCTION_ARG_KV_COLUMNS, KeyValue.COMPARATOR);
     }
-    
+
     private static final int IS_ARRAY_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(IS_ARRAY_KV);
     private static final int IS_CONSTANT_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(IS_CONSTANT_KV);
     private static final int DEFAULT_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(DEFAULT_VALUE_KV);
@@ -553,7 +572,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
 
             long currentTime = EnvironmentEdgeManager.currentTimeMillis();
-            PTable table = doGetTable(key, request.getClientTimestamp(), request.getClientVersion());
+            PTable table =
+                    doGetTable(tenantId, schemaName, tableName, request.getClientTimestamp(),
+                        null, request.getClientVersion(), request.getSkipAddingIndexes(),
+                        request.getSkipAddingParentColumns(),
+                        PTableImpl.createFromProto(request.getLockedAncestorTable()));
             if (table == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                 builder.setMutationTime(currentTime);
@@ -586,12 +609,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
                 }
             }
-
-            if (table.getTimeStamp() != tableTimeStamp) {
+            // the PTable of views and indexes on views might get updated because a column is added to one of
+            // their parents (this won't change the timestamp)
+            if (table.getType()!=PTableType.TABLE || table.getTimeStamp() != tableTimeStamp) {
                 builder.setTable(PTableImpl.toProto(table));
             }
             done.run(builder.build());
-            return;
         } catch (Throwable t) {
             logger.error("getTable failed", t);
             ProtobufUtil.setControllerException(controller,
@@ -599,35 +622,349 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
+    /**
+     * Used to add the columns present the ancestor hierarchy to the PTable of the given view or
+     * view index
+     * @param table PTable of the view or view index
+     * @param skipAddingIndexes if true the returned PTable won't include indexes
+     * @param skipAddingParentColumns if true the returned PTable won't include columns derived from
+     *            ancestor tables
+     * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to
+     *            resolve this table as its locked)
+     */
+    private Pair<PTable, MetaDataProtos.MutationCode> combineColumns(PTable table, long timestamp,
+            int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns,
+            PTable lockedAncestorTable) throws SQLException, IOException {
+        boolean hasIndexId = table.getViewIndexId() != null;
+        if (table.getType() != PTableType.VIEW && !hasIndexId) {
+            return new Pair<PTable, MetaDataProtos.MutationCode>(table,
+                    MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+        }
+        if (!skipAddingParentColumns) {
+            table =
+                    addDerivedColumnsFromAncestors(table, timestamp, clientVersion,
+                        lockedAncestorTable);
+            if (table==null) {
+                return new Pair<PTable, MetaDataProtos.MutationCode>(table,
+                        MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+            }
+            // we need to resolve the indexes of views (to get ensure they also have all the columns
+            // derived from their ancestors) 
+            if (!skipAddingIndexes && !table.getIndexes().isEmpty()) {
+                List<PTable> indexes = Lists.newArrayListWithExpectedSize(table.getIndexes().size());
+                for (PTable index : table.getIndexes()) {
+                    byte[] tenantIdBytes =
+                            index.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY
+                                    : index.getTenantId().getBytes();
+                    PTable latestIndex =
+                            doGetTable(tenantIdBytes, index.getSchemaName().getBytes(),
+                                index.getTableName().getBytes(), timestamp, null, clientVersion, true,
+                                false, lockedAncestorTable);
+                    if (latestIndex == null) {
+                        throw new TableNotFoundException(
+                                "Could not find index table while combining columns "
+                                        + index.getTableName().getString() + " with tenant id "
+                                        + index.getTenantId());
+                    }
+                    indexes.add(latestIndex);
+                }
+                table = PTableImpl.makePTable(table, table.getTimeStamp(), indexes);
+            }
+        }
+        
+        MetaDataProtos.MutationCode mutationCode =
+                table != null ? MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS
+                        : MetaDataProtos.MutationCode.TABLE_NOT_FOUND;
+        return new Pair<PTable, MetaDataProtos.MutationCode>(table, mutationCode);
+    }
+
+    
+    private PTable addDerivedColumnsFromAncestors(PTable table, long timestamp,
+            int clientVersion, PTable lockedAncestorTable) throws IOException, SQLException, TableNotFoundException {
+        // combine columns for view and view indexes
+        byte[] tenantId =
+                table.getTenantId() != null ? table.getTenantId().getBytes()
+                        : ByteUtil.EMPTY_BYTE_ARRAY;
+        byte[] schemaName = table.getSchemaName().getBytes();
+        byte[] tableName = table.getTableName().getBytes();
+        String fullTableName = SchemaUtil.getTableName(table.getSchemaName().getString(),
+                table.getTableName().getString());
+        boolean hasIndexId = table.getViewIndexId() != null;
+        boolean isSalted = table.getBucketNum() != null;
+        if (table.getType() != PTableType.VIEW && !hasIndexId) {
+            return table;
+        }
+        boolean isDiverged = isDivergedView(table);
+        // here you combine columns from the parent tables the logic is as follows, if the PColumn
+        // is in the EXCLUDED_COLUMNS remove it, otherwise priority of keeping duplicate columns is
+        // child -> parent
+        List<TableInfo> ancestorList = Lists.newArrayList();
+        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
+        if (PTableType.VIEW == table.getType()) {
+            findAncestorViews(tenantId, schemaName, tableName, viewFinderResult,
+                table.isNamespaceMapped());
+        } else { // is a view index
+            findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult,
+                table.isNamespaceMapped());
+        }
+        if (viewFinderResult.getLinks().isEmpty()) {
+            // no need to combine columns for local indexes on regular tables
+            return table;
+        }
+        for (TableInfo viewInfo : viewFinderResult.getLinks()) {
+            ancestorList.add(viewInfo);
+        }
+        List<PColumn> allColumns = Lists.newArrayList();
+        List<PColumn> excludedColumns = Lists.newArrayList();
+        // add my own columns first in reverse order
+        List<PColumn> myColumns = table.getColumns();
+        // skip salted column as it will be added from the base table columns
+        int startIndex = table.getBucketNum() != null ? 1 : 0;
+        for (int i = myColumns.size() - 1; i >= startIndex; i--) {
+            PColumn pColumn = myColumns.get(i);
+            if (pColumn.isExcluded()) {
+                excludedColumns.add(pColumn);
+            }
+            allColumns.add(pColumn);
+        }
+
+        // initialize map from with indexed expression to list of required data columns
+        // then remove the data columns that have not been dropped, so that we get the columns that
+        // have been dropped
+        Map<PColumn, List<String>> indexRequiredDroppedDataColMap =
+                Maps.newHashMapWithExpectedSize(table.getColumns().size());
+        if (hasIndexId) {
+            int indexPosOffset = (isSalted ? 1 : 0) + (table.isMultiTenant() ? 1 : 0) + 1;
+            ColumnNameTrackingExpressionCompiler expressionCompiler =
+                    new ColumnNameTrackingExpressionCompiler();
+            for (int i = indexPosOffset; i < table.getPKColumns().size(); i++) {
+                PColumn indexColumn = table.getPKColumns().get(i);
+                try {
+                    expressionCompiler.reset();
+                    String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
+                    ParseNode parseNode = SQLParser.parseCondition(expressionStr);
+                    parseNode.accept(expressionCompiler);
+                    indexRequiredDroppedDataColMap.put(indexColumn,
+                        Lists.newArrayList(expressionCompiler.getDataColumnNames()));
+                } catch (SQLException e) {
+                    throw new RuntimeException(e); // Impossible
+                }
+            }
+        }
+
+        // now go up from child to parent all the way to the base table:
+        PTable baseTable = null;
+        long maxTableTimestamp = -1;
+        int numPKCols = table.getPKColumns().size();
+        for (int i = 0; i < ancestorList.size(); i++) {
+            TableInfo parentTableInfo = ancestorList.get(i);
+            PTable pTable = null;
+            String fullParentTableName = SchemaUtil.getTableName(parentTableInfo.getSchemaName(),
+                parentTableInfo.getTableName());
+            PName parentTenantId =
+                    (parentTableInfo.getTenantId() != null && parentTableInfo.getTenantId().length!=0)
+                            ? PNameFactory.newName(parentTableInfo.getTenantId()) : null;
+            PTableKey pTableKey = new PTableKey(parentTenantId, fullParentTableName);
+            // if we already have the PTable of an ancestor that has been locked, no need to look up
+            // the table
+            if (lockedAncestorTable != null && lockedAncestorTable.getKey().equals(pTableKey)) {
+                pTable = lockedAncestorTable;
+            } else {
+                // if we are currently combining columns for a view index and are looking up its
+                // ancestors we do not add the indexes to the ancestor PTable (or else we end up in
+                // a circular loop)
+                // we also don't need to add parent columns of the ancestors as we combine columns
+                // from all ancestors
+                pTable =
+                        doGetTable(parentTableInfo.getTenantId(), parentTableInfo.getSchemaName(),
+                            parentTableInfo.getTableName(), timestamp, null, clientVersion, hasIndexId,
+                            true, null);
+            }
+            if (pTable == null) {
+                throw new ParentTableNotFoundException(parentTableInfo, fullTableName);
+            } else {
+                // only combine columns for view indexes (and not local indexes on regular tables
+                // which also have a viewIndexId)
+                if (i == 0 && hasIndexId && pTable.getType() != PTableType.VIEW) {
+                    return table;
+                }
+                if (TABLE.equals(pTable.getType())) {
+                    baseTable = pTable;
+                }
+                // set the final table timestamp as the max timestamp of the view/view index or its
+                // ancestors
+                maxTableTimestamp = Math.max(maxTableTimestamp, pTable.getTimeStamp());
+                if (hasIndexId) {
+                    // add all pk columns of parent tables to indexes
+                    // skip salted column as it will be added from the base table columns
+                    startIndex = pTable.getBucketNum() != null ? 1 : 0;
+                    for (int index=startIndex; index<pTable.getPKColumns().size(); index++) {
+                        PColumn column = pTable.getPKColumns().get(index);
+                        // don't add the salt column of ancestor tables for view indexes
+                        if (column.equals(SaltingUtil.SALTING_COLUMN) || column.isExcluded()) {
+                            continue;
+                        }
+                        column = IndexUtil.getIndexPKColumn(++numPKCols, column);
+                        int existingColumnIndex = allColumns.indexOf(column);
+                        if (existingColumnIndex == -1) {
+                            allColumns.add(0, column);
+                        }
+                    }
+                    for (int j = 0; j < pTable.getColumns().size(); j++) {
+                        PColumn tableColumn = pTable.getColumns().get(j);
+                        if (tableColumn.isExcluded()) {
+                            continue;
+                        }
+                        String dataColumnName = tableColumn.getName().getString();
+                        // remove from list of columns since it has not been dropped
+                        for (Entry<PColumn, List<String>> entry : indexRequiredDroppedDataColMap
+                                .entrySet()) {
+                            entry.getValue().remove(dataColumnName);
+                        }
+                    }
+                } else {
+                    List<PColumn> currAncestorTableCols = PTableImpl.getColumnsToClone(pTable);
+                    if (currAncestorTableCols != null) {
+                        for (int j = currAncestorTableCols.size() - 1; j >= 0; j--) {
+                            PColumn column = currAncestorTableCols.get(j);
+                            // for diverged views we always include pk columns of the base table. We
+                            // have to include these pk columns to be able to support adding pk
+                            // columns to the diverged view
+                            // we only include regular columns that were created before the view
+                            // diverged
+                            if (isDiverged && column.getFamilyName() != null
+                                    && column.getTimestamp() > table.getTimeStamp()) {
+                                continue;
+                            }
+                            // need to check if this column is in the list of excluded (dropped)
+                            // columns of the view
+                            int existingIndex = excludedColumns.indexOf(column);
+                            if (existingIndex != -1) {
+                                // if it is, only exclude the column if was created before the
+                                // column was dropped in the view in order to handle the case where
+                                // a base table column is dropped in a view, then dropped in the
+                                // base table and then added back to the base table
+                                if (column.getTimestamp() <= excludedColumns.get(existingIndex)
+                                        .getTimestamp()) {
+                                    continue;
+                                }
+                            }
+                            if (column.isExcluded()) {
+                                excludedColumns.add(column);
+                            } else {
+                                int existingColumnIndex = allColumns.indexOf(column);
+                                if (existingColumnIndex != -1) {
+                                    // if the same column exists in a parent and child, we keep the
+                                    // latest column
+                                    PColumn existingColumn = allColumns.get(existingColumnIndex);
+                                    if (column.getTimestamp() > existingColumn.getTimestamp()) {
+                                        allColumns.remove(existingColumnIndex);
+                                        allColumns.add(column);
+                                    }
+                                } else {
+                                    allColumns.add(column);
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        // at this point indexRequiredDroppedDataColMap only contain the columns required by a view
+        // index that have dropped
+        for (Entry<PColumn, List<String>> entry : indexRequiredDroppedDataColMap.entrySet()) {
+            if (!entry.getValue().isEmpty()) {
+                PColumn indexColumnToBeDropped = entry.getKey();
+                if (SchemaUtil.isPKColumn(indexColumnToBeDropped)) {
+                    // if an indexed column was dropped in an ancestor then we
+                    // cannot use this index an more
+                    // TODO figure out a way to actually drop this view index
+                    return null;
+                } else {
+                    allColumns.remove(indexColumnToBeDropped);
+                }
+            }
+        }
+        // lets remove the excluded columns first if the timestamp is newer than
+        // the added column
+        for (PColumn excludedColumn : excludedColumns) {
+            int index = allColumns.indexOf(excludedColumn);
+            if (index != -1) {
+                if (allColumns.get(index).getTimestamp() <= excludedColumn.getTimestamp()) {
+                    allColumns.remove(excludedColumn);
+                }
+            }
+        }
+        List<PColumn> columnsToAdd = Lists.newArrayList();
+        int position = isSalted ? 1 : 0;
+        for (int i = allColumns.size() - 1; i >= 0; i--) {
+            PColumn column = allColumns.get(i);
+            if (table.getColumns().contains(column)) {
+                // for views this column is not derived from an ancestor
+                columnsToAdd.add(new PColumnImpl(column, position));
+            } else {
+                columnsToAdd.add(new PColumnImpl(column, true, position));
+            }
+            position++;
+        }
+        // need to have the columns in the PTable to use the WhereCompiler
+        // unfortunately so this needs to be done
+        // twice....
+        // TODO set the view properties correctly instead of just setting them
+        // same as the base table
+        int baseTableColumnCount =
+                isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT
+                        : columnsToAdd.size() - myColumns.size();
+        PTableImpl pTable =
+                PTableImpl.makePTable(table, baseTable, columnsToAdd, maxTableTimestamp,
+                    baseTableColumnCount);
+        return WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
+    }
+
     private PhoenixMetaDataCoprocessorHost getCoprocessorHost() {
         return phoenixAccessCoprocessorHost;
     }
 
+    /**
+     * @param skipAddingIndexes if true the PTable will not include indexes for tables or views
+     * @param skipAddingParentColumns if true the PTable will not include parent columns for views
+     *            or indexes
+     * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to
+     *            resolve this table as its locked)
+     */
     private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
-            long clientTimeStamp, int clientVersion) throws IOException, SQLException {
+            long clientTimeStamp, int clientVersion, boolean skipAddingIndexes,
+            boolean skipAddingParentColumns, PTable lockedAncestorTable)
+            throws IOException, SQLException {
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
         Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        PTable newTable;
         try (RegionScanner scanner = region.getScanner(scan)) {
             PTable oldTable = (PTable)metaDataCache.getIfPresent(cacheKey);
             long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP-1 : oldTable.getTimeStamp();
-            PTable newTable;
-            newTable = getTable(scanner, clientTimeStamp, tableTimeStamp, clientVersion);
-            if (newTable == null) {
-                return null;
-            }
-            if (oldTable == null || tableTimeStamp < newTable.getTimeStamp()
-                    || (blockWriteRebuildIndex && newTable.getIndexDisableTimestamp() > 0)) {
+            newTable = getTable(scanner, clientTimeStamp, tableTimeStamp, clientVersion, skipAddingIndexes, skipAddingParentColumns);
+            if (newTable != null
+                    && (oldTable == null || tableTimeStamp < newTable.getTimeStamp()
+                            || (blockWriteRebuildIndex && newTable.getIndexDisableTimestamp() > 0))
+                    // only cache the PTable if it has the required indexes,
+                    // the PTable added to the cache doesn't include parent columns as we always call 
+                    // combine columns after looking up the PTable from the cache
+                    && !skipAddingIndexes) {
                 if (logger.isDebugEnabled()) {
                     logger.debug("Caching table "
                             + Bytes.toStringBinary(cacheKey.get(), cacheKey.getOffset(),
-                                cacheKey.getLength()) + " at seqNum "
-                            + newTable.getSequenceNumber() + " with newer timestamp "
-                            + newTable.getTimeStamp() + " versus " + tableTimeStamp);
+                                cacheKey.getLength())
+                            + " at seqNum " + newTable.getSequenceNumber()
+                            + " with newer timestamp " + newTable.getTimeStamp() + " versus "
+                            + tableTimeStamp);
                 }
                 metaDataCache.put(cacheKey, newTable);
             }
-            return newTable;
+            if (newTable != null) {
+                newTable = combineColumns(newTable, clientTimeStamp, clientVersion, skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable).getFirst();
+            }
         }
+        return newTable;
     }
 
     private List<PFunction> buildFunctions(List<byte[]> keys, Region region,
@@ -694,9 +1031,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private void addIndexToTable(PName tenantId, PName schemaName, PName indexName, PName tableName, long clientTimeStamp, List<PTable> indexes, int clientVersion) throws IOException, SQLException {
-        byte[] key = SchemaUtil.getTableKey(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes(), schemaName.getBytes(), indexName.getBytes());
-        PTable indexTable = doGetTable(key, clientTimeStamp, clientVersion);
+    private void addIndexToTable(PName tenantId, PName schemaName, PName indexName, PName tableName,
+            long clientTimeStamp, List<PTable> indexes, int clientVersion, boolean skipAddingParentColumns)
+            throws IOException, SQLException {
+        byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
+        PTable indexTable = doGetTable(tenantIdBytes, schemaName.getBytes(), indexName.getBytes(), clientTimeStamp,
+                null, clientVersion, false, skipAddingParentColumns, null);
         if (indexTable == null) {
             ServerUtil.throwIOException("Index not found", new TableNotFoundException(schemaName.getString(), indexName.getString()));
             return;
@@ -704,6 +1044,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         indexes.add(indexTable);
     }
 
+    private void addExcludedColumnToTable(List<PColumn> pColumns, PName colName, PName famName, long timestamp) {
+        PColumnImpl pColumn = PColumnImpl.createExcludedColumn(famName, colName, timestamp);
+        pColumns.add(pColumn);
+    }
+
     private void addColumnToTable(List<Cell> results, PName colName, PName famName,
         Cell[] colKeyValues, List<PColumn> columns, boolean isSalted) {
         int i = 0;
@@ -775,7 +1120,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 isRowTimestampKV == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
                         isRowTimestampKV.getValueArray(), isRowTimestampKV.getValueOffset(),
                         isRowTimestampKV.getValueLength()));
-        
+
         boolean isPkColumn = famName == null || famName.getString() == null;
         Cell columnQualifierKV = colKeyValues[COLUMN_QUALIFIER_INDEX];
         // Older tables won't have column qualifier metadata present. To make things simpler, just set the
@@ -784,10 +1129,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 Arrays.copyOfRange(columnQualifierKV.getValueArray(),
                     columnQualifierKV.getValueOffset(), columnQualifierKV.getValueOffset()
                             + columnQualifierKV.getValueLength()) : (isPkColumn ? null : colName.getBytes());
-        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false, columnQualifierBytes);
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false, columnQualifierBytes,
+            results.get(0).getTimestamp());
         columns.add(column);
     }
-    
+
     private void addArgumentToFunction(List<Cell> results, PName functionName, PName type,
         Cell[] functionKeyValues, List<FunctionArgument> arguments, short argPosition) throws SQLException {
         int i = 0;
@@ -843,7 +1189,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         arguments.add(arg);
     }
 
-    private PTable getTable(RegionScanner scanner, long clientTimeStamp, long tableTimeStamp, int clientVersion)
+    /**
+     * @param skipAddingIndexes if true the returned PTable for a table or view won't include indexes
+     * @param skipAddingParentColumns if true the returned PTable won't include inherited columns
+     * @return PTable 
+     */
+    private PTable getTable(RegionScanner scanner, long clientTimeStamp, long tableTimeStamp,
+            int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns)
         throws IOException, SQLException {
         List<Cell> results = Lists.newArrayList();
         scanner.next(results);
@@ -1054,15 +1406,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
           if (isQualifierCounterKV(colKv)) {
               Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
               cqCounter.setValue(famName.getString(), value);
-          } else if (Bytes.compareTo(LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length, colKv.getQualifierArray(), colKv.getQualifierOffset(), colKv.getQualifierLength())==0) {    
+          } else if (Bytes.compareTo(LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length, colKv.getQualifierArray(), colKv.getQualifierOffset(), colKv.getQualifierLength())==0) {
               LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
-              if (linkType == LinkType.INDEX_TABLE) {
-                  addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion);
+              if (linkType == LinkType.INDEX_TABLE && !skipAddingIndexes) {
+                  addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion, skipAddingParentColumns);
               } else if (linkType == LinkType.PHYSICAL_TABLE) {
                   physicalTables.add(famName);
               } else if (linkType == LinkType.PARENT_TABLE) {
                   parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
                   parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
+              } else if (linkType == LinkType.EXCLUDED_COLUMN) {
+                  // add the excludedColumn
+                  addExcludedColumnToTable(columns, colName, famName, colKv.getTimestamp());
               }
           } else {
               addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null);
@@ -1076,7 +1431,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, baseColumnCount,
                 indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, encodingScheme, cqCounter, useStatsForParallelization);
     }
-    
+
     private boolean isQualifierCounterKV(Cell kv) {
         int cmp =
                 Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
@@ -1084,7 +1439,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     QUALIFIER_COUNTER_KV.getQualifierOffset(), QUALIFIER_COUNTER_KV.getQualifierLength());
         return cmp == 0;
     }
-    
+
     private PSchema getSchema(RegionScanner scanner, long clientTimeStamp) throws IOException, SQLException {
         List<Cell> results = Lists.newArrayList();
         scanner.next(results);
@@ -1203,7 +1558,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             PName typeName =
                     newPName(typeKv.getRowArray(), typeKv.getRowOffset() + offset, typeKeyLength
                             - offset - 3);
-            
+
             int argPositionOffset =  offset + typeName.getBytes().length + 1;
             short argPosition = Bytes.toShort(typeKv.getRowArray(), typeKv.getRowOffset() + argPositionOffset, typeKeyLength
                 - argPositionOffset);
@@ -1218,7 +1573,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return new PFunction(tenantId, functionName.getString(), arguments, returnType.getString(),
                 className.getString(), jarPath == null ? null : jarPath.getString(), timeStamp);
     }
-    
+
     private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
         long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
@@ -1245,7 +1600,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return null;
     }
 
-    
+
     private PFunction buildDeletedFunction(byte[] key, ImmutableBytesPtr cacheKey, Region region,
         long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
@@ -1322,21 +1677,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return function.getFunctionName() == null;
     }
 
-    private PTable getTable(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey,
-            long clientTimeStamp, long asOfTimeStamp, int clientVersion) throws IOException, SQLException {
-        PTable table = loadTable(env, key, cacheKey, clientTimeStamp, asOfTimeStamp, clientVersion);
-        if (table == null || isTableDeleted(table)) { return null; }
-        return table;
-    }
-
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
         throws IOException, SQLException {
         Region region = env.getRegion();
-        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-        PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
+        PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion, false, false, null);
         // We always cache the latest version - fault in if not in cache
-        if (table != null || (table = buildTable(key, cacheKey, region, asOfTimeStamp, clientVersion)) != null) {
+        if (table != null || (table = buildTable(key, cacheKey, region, asOfTimeStamp, clientVersion, false, false, null)) != null) {
             return table;
         }
         // if not found then check if newer table already exists and add delete marker for timestamp
@@ -1347,6 +1694,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         return null;
     }
+    
+    /**
+     * Returns a PTable if its found in the cache.
+     */
+   private PTable getTableFromCache(ImmutableBytesPtr cacheKey, long clientTimeStamp, int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns, PTable lockedAncestorTable) throws SQLException, IOException {
+       Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+       PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
+       if (table!=null)
+           table = combineColumns(table, clientTimeStamp, clientVersion, skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable).getFirst();
+       return table;
+   }
 
     private PFunction loadFunction(RegionCoprocessorEnvironment env, byte[] key,
             ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace)
@@ -1390,9 +1748,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     /**
-     * 
      * @return null if the physical table row information is not present.
-     * 
      */
     private static Mutation getPhysicalTableRowForView(List<Mutation> tableMetadata, byte[][] parentTenantSchemaTableNames, byte[][] physicalSchemaTableNames) {
         int size = tableMetadata.size();
@@ -1457,7 +1813,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             schemaTableNames[2] = tName;
         }
     }
-    
+
     @Override
     public void createTable(RpcController controller, CreateTableRequest request,
             RpcCallback<MetaDataResponse> done) {
@@ -1465,6 +1821,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         byte[][] rowKeyMetaData = new byte[3][];
         byte[] schemaName = null;
         byte[] tableName = null;
+        String fullTableName = null;
         try {
             int clientVersion = request.getClientVersion();
             List<Mutation> tableMetadata = ProtobufUtil.getMutations(request);
@@ -1472,13 +1829,92 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+            fullTableName = SchemaUtil.getTableName(schemaName, tableName);
             boolean isNamespaceMapped = MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
                     new ImmutableBytesWritable());
             final IndexType indexType = MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
                     new ImmutableBytesWritable());
+            byte[] parentTenantId = null;
             byte[] parentSchemaName = null;
             byte[] parentTableName = null;
             PTableType tableType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+            ViewType viewType = MetaDataUtil.getViewType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+
+            // Load table to see if it already exists
+            byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
+            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
+            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+            PTable table = null;
+            try {
+                // Get as of latest timestamp so we can detect if we have a newer table that already
+                // exists without making an additional query
+                table = loadTable(env, tableKey, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP,
+                        clientVersion);
+            } catch (ParentTableNotFoundException e) {
+                dropChildMetadata(e.getParentSchemaName(), e.getParentTableName(), e.getParentTenantId());
+            }
+            if (table != null) {
+                if (table.getTimeStamp() < clientTimeStamp) {
+                    // If the table is older than the client time stamp and it's deleted,
+                    // continue
+                    if (!isTableDeleted(table)) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        builder.setTable(PTableImpl.toProto(table));
+                        done.run(builder.build());
+                        return;
+                    }
+                } else {
+                    builder.setReturnCode(MetaDataProtos.MutationCode.NEWER_TABLE_FOUND);
+                    builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                    builder.setTable(PTableImpl.toProto(table));
+                    done.run(builder.build());
+                    return;
+                }
+            }
+            
+            // check if the table was dropped, but had child views that were have not yet
+            // been cleaned up by compaction
+            if (!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                dropChildMetadata(schemaName, tableName, tenantIdBytes);
+            }
+            
+            // Here we are passed the parent's columns to add to a view, PHOENIX-3534 allows for a splittable
+            // System.Catalog thus we only store the columns that are new to the view, not the parents columns,
+            // thus here we remove everything that is ORDINAL.POSITION <= baseColumnCount and update the
+            // ORDINAL.POSITIONS to be shifted accordingly.
+            // TODO PHOENIX-4767 remove the following code that removes the base table column metadata in the next release 
+            if (PTableType.VIEW.equals(tableType) && !ViewType.MAPPED.equals(viewType)) {
+                boolean isSalted = MetaDataUtil.getSaltBuckets(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable()) > 0;
+                int baseColumnCount = MetaDataUtil.getBaseColumnCount(tableMetadata) - (isSalted ? 1 : 0);
+                if (baseColumnCount > 0) {
+                    Iterator<Mutation> mutationIterator = tableMetadata.iterator();
+                    while (mutationIterator.hasNext()) {
+                        Mutation mutation = mutationIterator.next();
+                        // if not null and ordinal position < base column count remove this mutation
+                        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                        MetaDataUtil.getMutationValue(mutation, PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES,
+                            GenericKeyValueBuilder.INSTANCE, ptr);
+                        if (MetaDataUtil.getMutationValue(mutation, PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES,
+                            GenericKeyValueBuilder.INSTANCE, ptr)) {
+                            int ordinalValue = PInteger.INSTANCE.getCodec().decodeInt(ptr, SortOrder.ASC);
+                            if (ordinalValue <= baseColumnCount) {
+                                mutationIterator.remove();
+                            } else {
+                                if (mutation instanceof Put) {
+                                    byte[] ordinalPositionBytes = new byte[PInteger.INSTANCE.getByteSize()];
+                                    int newOrdinalValue = ordinalValue - baseColumnCount;
+                                    PInteger.INSTANCE.getCodec()
+                                        .encodeInt(newOrdinalValue, ordinalPositionBytes, 0);
+                                    byte[] family = Iterables.getOnlyElement(mutation.getFamilyCellMap().keySet());
+                                    MetaDataUtil.mutatePutValue((Put) mutation, family, PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES, ordinalPositionBytes);
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+
             byte[] parentTableKey = null;
             Mutation viewPhysicalTableRow = null;
             Set<TableName> indexes = new HashSet<TableName>();;
@@ -1489,19 +1925,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 byte[][] parentSchemaTableNames = new byte[3][];
                 byte[][] parentPhysicalSchemaTableNames = new byte[3][];
                 /*
-                 * For a view, we lock the base physical table row. For a mapped view, there is 
-                 * no link present to the physical table. So the viewPhysicalTableRow is null
-                 * in that case.
+                 * For a mapped view, there is no link present to the physical table. So the
+                 * viewPhysicalTableRow is null in that case.
                  */
                 
                 viewPhysicalTableRow = getPhysicalTableRowForView(tableMetadata, parentSchemaTableNames,parentPhysicalSchemaTableNames);
-                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 if (parentPhysicalSchemaTableNames[2] != null) {
                     
                     parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                             parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
-                    PTable parentTable = getTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
-                            clientTimeStamp, clientTimeStamp, clientVersion);
+                    PTable parentTable =
+                            doGetTable(ByteUtil.EMPTY_BYTE_ARRAY, parentPhysicalSchemaTableNames[1],
+                                parentPhysicalSchemaTableNames[2], clientTimeStamp, clientVersion);
                     if (parentTable == null) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
                         builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
@@ -1512,17 +1947,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (parentSchemaTableNames[2] != null
                             && Bytes.compareTo(parentSchemaTableNames[2], parentPhysicalSchemaTableNames[2]) != 0) {
                         // if view is created on view
-                        byte[] parentKey = SchemaUtil.getTableKey(
-                                parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
-                                parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                        parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
-                                clientTimeStamp, clientTimeStamp, clientVersion);
+                        byte[] tenantId =
+                                parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY
+                                        : parentSchemaTableNames[0];
+                        parentTable =
+                                doGetTable(tenantId, parentSchemaTableNames[1],
+                                    parentSchemaTableNames[2], clientTimeStamp, clientVersion);
                         if (parentTable == null) {
                             // it could be a global view
-                            parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
-                                    parentSchemaTableNames[1], parentSchemaTableNames[2]);
-                            parentTable = getTable(env, parentKey, new ImmutableBytesPtr(parentKey),
-                                    clientTimeStamp, clientTimeStamp, clientVersion);
+                            parentTable =
+                                    doGetTable(ByteUtil.EMPTY_BYTE_ARRAY, parentSchemaTableNames[1],
+                                        parentSchemaTableNames[2], clientTimeStamp, clientVersion);
                         }
                     }
                     if (parentTable == null) {
@@ -1538,6 +1973,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     // Mapped View
                     cParentPhysicalName = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
                 }
+                parentTenantId = ByteUtil.EMPTY_BYTE_ARRAY;
                 parentSchemaName = parentPhysicalSchemaTableNames[1];
                 parentTableName = parentPhysicalSchemaTableNames[2];
                     
@@ -1547,12 +1983,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                  * For an index we lock the parent table's row which could be a physical table or a view.
                  * If the parent table is a physical table, then the tenantIdBytes is empty because
                  * we allow creating an index with a tenant connection only if the parent table is a view.
-                 */ 
+                 */
+                parentTenantId = tenantIdBytes;
                 parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
                 parentTableKey = SchemaUtil.getTableKey(tenantIdBytes, parentSchemaName, parentTableName);
-                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-                PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
-                        clientTimeStamp, clientTimeStamp, clientVersion);
+                PTable parentTable =
+                        doGetTable(tenantIdBytes, parentSchemaName, parentTableName, clientTimeStamp, null,
+                            request.getClientVersion(), false, false, null);
                 if (IndexType.LOCAL == indexType) {
                     cPhysicalName = parentTable.getPhysicalName().getBytes();
                     cParentPhysicalName=parentTable.getPhysicalName().getBytes();
@@ -1566,7 +2003,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             
             getCoprocessorHost().preCreateTable(Bytes.toString(tenantIdBytes),
-                    SchemaUtil.getTableName(schemaName, tableName),
+                    fullTableName,
                     (tableType == PTableType.VIEW) ? null : TableName.valueOf(cPhysicalName),
                     cParentPhysicalName == null ? null : TableName.valueOf(cParentPhysicalName), tableType,
                     /* TODO: During inital create we may not need the family map */
@@ -1575,7 +2012,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             Region region = env.getRegion();
             List<RowLock> locks = Lists.newArrayList();
             // Place a lock using key for the table to be created
-            byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
             try {
                 acquireLock(region, tableKey, locks);
 
@@ -1586,74 +2022,52 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return;
                 }
 
-                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 ImmutableBytesPtr parentCacheKey = null;
                 PTable parentTable = null;
                 if (parentTableName != null) {
-                    // Check if the parent table resides in the same region. If not, don't worry about locking the parent table row
-                    // or loading the parent table. For a view, the parent table that needs to be locked is the base physical table.
-                    // For an index on view, the view header row needs to be locked. 
-                    result = checkTableKeyInRegion(parentTableKey, region);
-                    if (result == null) {
-                        acquireLock(region, parentTableKey, locks);
-                        parentCacheKey = new ImmutableBytesPtr(parentTableKey);
-                        parentTable = loadTable(env, parentTableKey, parentCacheKey, clientTimeStamp,
-                                clientTimeStamp, clientVersion);
-                        if (parentTable == null || isTableDeleted(parentTable)) {
-                            builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
-                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                            done.run(builder.build());
-                            return;
-                        }
-                        // make sure we haven't gone over our threshold for indexes on this table.
-                        if (execeededIndexQuota(tableType, parentTable)) {
-                            builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
-                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                            done.run(builder.build());
-                            return;
-                        }
-                        long parentTableSeqNumber;
-                        if (tableType == PTableType.VIEW && viewPhysicalTableRow != null && request.hasClientVersion()) {
-                            // Starting 4.5, the client passes the sequence number of the physical table in the table metadata.
-                            parentTableSeqNumber = MetaDataUtil.getSequenceNumber(viewPhysicalTableRow);
-                        } else if (tableType == PTableType.VIEW && !request.hasClientVersion()) {
-                            // Before 4.5, due to a bug, the parent table key wasn't available.
-                            // So don't do anything and prevent the exception from being thrown.
-                            parentTableSeqNumber = parentTable.getSequenceNumber();
-                        } else {
-                            parentTableSeqNumber = MetaDataUtil.getParentSequenceNumber(tableMetadata);
-                        }
-                        // If parent table isn't at the expected sequence number, then return
-                        if (parentTable.getSequenceNumber() != parentTableSeqNumber) {
-                            builder.setReturnCode(MetaDataProtos.MutationCode.CONCURRENT_TABLE_MUTATION);
+                    // we lock the parent table when creating an index on a table or a view
+                    if (tableType == PTableType.INDEX) {
+                        result = checkTableKeyInRegion(parentTableKey, region);
+                        if (result != null) {
+                            builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_IN_REGION);
                             builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                            builder.setTable(PTableImpl.toProto(parentTable));
                             done.run(builder.build());
                             return;
                         }
+                        acquireLock(region, parentTableKey, locks);
                     }
-                }
-                // Load child table next
-                ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
-                // Get as of latest timestamp so we can detect if we have a newer table that already
-                // exists without making an additional query
-                PTable table =
-                        loadTable(env, tableKey, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP, clientVersion);
-                if (table != null) {
-                    if (table.getTimeStamp() < clientTimeStamp) {
-                        // If the table is older than the client time stamp and it's deleted,
-                        // continue
-                        if (!isTableDeleted(table)) {
-                            builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
-                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                            builder.setTable(PTableImpl.toProto(table));
-                            done.run(builder.build());
-                            return;
-                        }
+                    parentTable =  doGetTable(parentTenantId, parentSchemaName, parentTableName,
+                        clientTimeStamp, null, clientVersion, false, false, null);
+                    parentCacheKey = new ImmutableBytesPtr(parentTableKey);
+                    if (parentTable == null || isTableDeleted(parentTable)) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    // make sure we haven't gone over our threshold for indexes on this table.
+                    if (execeededIndexQuota(tableType, parentTable)) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    long parentTableSeqNumber;
+                    if (tableType == PTableType.VIEW && viewPhysicalTableRow != null && request.hasClientVersion()) {
+                        // Starting 4.5, the client passes the sequence number of the physical table in the table metadata.
+                        parentTableSeqNumber = MetaDataUtil.getSequenceNumber(viewPhysicalTableRow);
+                    } else if (tableType == PTableType.VIEW && !request.hasClientVersion()) {
+                        // Before 4.5, due to a bug, the parent table key wasn't available.
+                        // So don't do anything and prevent the exception from being thrown.
+                        parentTableSeqNumber = parentTable.getSequenceNumber();
                     } else {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.NEWER_TABLE_FOUND);
+                        parentTableSeqNumber = MetaDataUtil.getParentSequenceNumber(tableMetadata);
+                    }
+                    // If parent table isn't at the expected sequence number, then return
+                    if (parentTable.getSequenceNumber() != parentTableSeqNumber) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.CONCURRENT_TABLE_MUTATION);
                         builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        builder.setTable(PTableImpl.toProto(table));
+                        builder.setTable(PTableImpl.toProto(parentTable));
                         done.run(builder.build());
                         return;
                     }
@@ -1665,7 +2079,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (tableType != PTableType.VIEW) {
                     UpgradeUtil.addRowKeyOrderOptimizableCell(tableMetadata, tableKey, clientTimeStamp);
                 }
-                // If the parent table of the view has the auto partition sequence name attribute, modify the 
+                // If the parent table of the view has the auto partition sequence name attribute, modify the
                 // tableMetadata and set the view statement and partition column correctly
                 if (parentTable!=null && parentTable.getAutoPartitionSeqName()!=null) {
                     long autoPartitionNum = 1;
@@ -1696,7 +2110,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                     builder.setAutoPartitionNum(autoPartitionNum);
-                    
+
                     // set the VIEW STATEMENT column of the header row
                     Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
                     NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
@@ -1713,13 +2127,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (!Bytes.equals(value, QueryConstants.EMPTY_COLUMN_VALUE_BYTES)) {
                         viewStatement = Bytes.add(value, Bytes.toBytes(" AND "), Bytes.toBytes(autoPartitionWhere));
                     }
-                    else { 
+                    else {
                         viewStatement = Bytes.toBytes(QueryUtil.getViewStatement(parentTable.getSchemaName().getString(), parentTable.getTableName().getString(), autoPartitionWhere));
                     }
                     Cell viewStatementCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_STATEMENT_BYTES,
                         cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), viewStatement);
                     cells.add(viewStatementCell);
-                    
+
                     // set the IS_VIEW_REFERENCED column of the auto partition column row
                     Put autoPartitionPut = MetaDataUtil.getPutOnlyAutoPartitionColumn(parentTable, tableMetadata);
                     familyCellMap = autoPartitionPut.getFamilyCellMap();
@@ -1743,7 +2157,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
                         // TODO Review Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
                         // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
-                        // or not. 
+                        // or not.
                         long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
                         try {
                             connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
@@ -1765,6 +2179,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             return;
                         }
                         Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
+
                         NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
                         List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
                         Cell cell = cells.get(0);
@@ -1779,13 +2194,66 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
                 }
                 
+                // The mutations to create a table are written in the following order:
+                // 1. Write the child link as if the next two steps fail we
+                // ignore missing children while processing a parent
+                // 2. Update the encoded column qualifier for the parent table if its on a
+                // different region server (for tables that use column qualifier encoding)
+                // if the next step fails we end up wasting a few col qualifiers
+                // 3. Finally write the mutations to create the table
+
+                // From 4.15 the parent->child links are stored in a separate table SYSTEM.CHILD_LINK
+                // TODO remove this after PHOENIX-4763 is implemented
+                List<Mutation> childLinkMutations = MetaDataUtil.removeChildLinks(tableMetadata);
+                MetaDataResponse response =
+                        processRemoteRegionMutations(
+                            PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                            childLinkMutations, fullTableName,
+                            MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
+                if (response != null) {
+                    done.run(response);
+                    return;
+                }
+
+                List<Mutation> localMutations =
+                        Lists.newArrayListWithExpectedSize(tableMetadata.size());
+                List<Mutation> remoteMutations = Lists.newArrayListWithExpectedSize(2);
+                // check to see if there are any mutations that should not be applied to this region
+                separateLocalAndRemoteMutations(region, tableMetadata, localMutations, remoteMutations);
+                if (!remoteMutations.isEmpty()) {
+                    // there should only be remote mutations if we are creating a view that uses
+                    // encoded column qualifiers (the remote mutations are to update the encoded
+                    // column qualifier counter on the parent table)
+                    if (parentTable != null && tableType == PTableType.VIEW && parentTable
+                            .getEncodingScheme() != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) {
+                        response =
+                                processRemoteRegionMutations(
+                                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
+                                    remoteMutations, fullTableName,
+                                    MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                        if (response != null) {
+                            done.run(response);
+                            return;
+                        }
+                    }
+                    else {
+                        String msg = "Found unexpected mutations while creating "+fullTableName;
+                        logger.error(msg);
+                        for (Mutation m : remoteMutations) {
+                            logger.debug("Mutation rowkey : " + Bytes.toStringBinary(m.getRow()));
+                            logger.debug("Mutation family cell map : " + m.getFamilyCellMap());
+                        }
+                        throw new IllegalStateException(msg);
+                    }
+                }
+                
                 // TODO: Switch this to HRegion#batchMutate when we want to support indexes on the
                 // system table. Basically, we get all the locks that we don't already hold for all the
                 // tableMetadata rows. This ensures we don't have deadlock situations (ensuring
                 // primary and then index table locks are held, in that order). For now, we just don't support
                 // indexing on the system table. This is an issue because of the way we manage batch mutation
                 // in the Indexer.
-                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, localMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getTable call will add it
                 // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
@@ -1809,180 +2277,97 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } catch (Throwable t) {
             logger.error("createTable failed", t);
             ProtobufUtil.setControllerException(controller,
-                    ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
+                    ServerUtil.createIOException(fullTableName, t));
         }
     }
 
-    private boolean execeededIndexQuota(PTableType tableType, PTable parentTable) {
-        return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= maxIndexesPerTable;
-    }
-    
-    private void findAllChildViews(Region region, byte[] tenantId, PTable table,
-            TableViewFinder result, long clientTimeStamp, int clientVersion) throws IOException, SQLException {
-        TableViewFinder currResult = findChildViews(region, tenantId, table, clientVersion, false);
-        result.addResult(currResult);
-        for (ViewInfo viewInfo : currResult.getViewInfoList()) {
-            byte[] viewtenantId = viewInfo.getTenantId();
-            byte[] viewSchema = viewInfo.getSchemaName();
-            byte[] viewTable = viewInfo.getViewName();
-            byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, viewTable);
-            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
-            PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, clientTimeStamp, clientVersion);
-            if (view == null) {
-                logger.warn("Found orphan tenant view row in SYSTEM.CATALOG with tenantId:"
-                        + Bytes.toString(tenantId) + ", schema:"
-                        + Bytes.toString(viewSchema) + ", table:"
-                        + Bytes.toString(viewTable));
-                continue;
+    private void dropChildMetadata(byte[] schemaName, byte[] tableName, byte[] tenantIdBytes)
+            throws IOException, SQLException, ClassNotFoundException {
+        TableViewFinderResult childViewsResult = new TableViewFinderResult();
+        findAllChildViews(tenantIdBytes, schemaName, tableName, childViewsResult);
+        if (childViewsResult.hasLinks()) {
+            for (TableInfo viewInfo : childViewsResult.getLinks()) {
+                byte[] viewTenantId = viewInfo.getTenantId();
+                byte[] viewSchemaName = viewInfo.getSchemaName();
+                byte[] viewName = viewInfo.getTableName();
+                Properties props = new Properties();
+                if (viewTenantId != null && viewTenantId.length != 0)
+                    props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(viewTenantId));
+                try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration())
+                        .unwrap(PhoenixConnection.class)) {
+                    MetaDataClient client = new MetaDataClient(connection);
+                    org.apache.phoenix.parse.TableName viewTableName = org.apache.phoenix.parse.TableName
+                            .create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
+                    client.dropTable(
+                            new DropTableStatement(viewTableName, PTableType.VIEW, false, true, true));
+                }
             }
-            findAllChildViews(region, viewtenantId, view, result, clientTimeStamp, clientVersion);
         }
     }
-        
-    // TODO use child link instead once splittable system catalog (PHOENIX-3534) is implemented
-    // and we have a separate table for links.
-    private TableViewFinder findChildViews_deprecated(Region region, byte[] tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) throws IOException {
-        byte[] schemaName = table.getSchemaName().getBytes();
-        byte[] tableName = table.getTableName().getBytes();
-        boolean isMultiTenant = table.isMultiTenant();
-        Scan scan = new Scan();
-        // If the table is multi-tenant, we need to check across all tenant_ids,
-        // so we can't constrain the row key. Otherwise, any views would have
-        // the same tenantId.
-        if (!isMultiTenant) {
-            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
-            byte[] stopRow = ByteUtil.nextKey(startRow);
-            scan.setStartRow(startRow);
-            scan.setStopRow(stopRow);
-        }
-        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
-        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
-                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
-        tableTypeFilter.setFilterIfMissing(false);
-        linkFilter.setFilterIfMissing(true);
-        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
-                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
-                .getBytes());
-        SuffixFilter rowFilter = new SuffixFilter(suffix);
-        List<Filter> filters = Lists.<Filter>newArrayList(linkFilter,tableTypeFilter,rowFilter);
-        if (stopAfterFirst) {
-            filters.add(new PageFilter(1));
-        }
-        FilterList filter = new FilterList(filters);
-        scan.setFilter(filter);
-        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
-        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
-        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
-        
-        // Original region-only scanner modified due to PHOENIX-1208
-        // RegionScanner scanner = region.getScanner(scan);
-        // The following *should* work, but doesn't due to HBASE-11837
-        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
-        // HTableInterface hTable = env.getTable(systemCatalogTableName);
-        // These deprecated calls work around the issue
-        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
-            region.getTableDesc().getTableName().getName())) {
-            boolean allViewsInCurrentRegion = true;
-            int numOfChildViews = 0;
-            List<ViewInfo> viewInfoList = Lists.newArrayList();
-            try (ResultScanner scanner = hTable.getScanner(scan)) {
-                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
-                    numOfChildViews++;
-                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-                    ResultTuple resultTuple = new ResultTuple(result);
-                    resultTuple.getKey(ptr);
-                    byte[] key = ptr.copyBytes();
-                    if (checkTableKeyInRegion(key, region) != null) {
-                        allViewsInCurrentRegion = false;
-                    }
-                    byte[][] rowKeyMetaData = new byte[3][];
-                    getVarChars(result.getRow(), 3, rowKeyMetaData);
-                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
-                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
-                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
-                }
-                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
-                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
-                    tableViewFinderResult.setAllViewsNotInSingleRegion();
-                }
-                return tableViewFinderResult;
+
+    private boolean execeededIndexQuota(PTableType tableType, PTable parentTable) {
+        return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= maxIndexesPerTable;
+    }
+
+    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName,
+            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
+        try (Table hTable =
+                env.getTable(SchemaUtil.getPhysicalTableName(
+                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
+            TableViewFinderResult currentResult =
+                    ViewFinder.findParentViewofIndex(hTable, tenantId, schemaName, indexName);
+            if (currentResult.getLinks().size() == 1) {
+                result.addResult(currentResult);
+                TableInfo tableInfo = currentResult.getLinks().get(0);
+                findAncestorViews(tableInfo.getTenantId(), tableInfo.getSchemaName(),
+                    tableInfo.getTableName(), result, isNamespaceMapped);
             }
+            // else this is an index on a regular table and so we don't need to combine columns
         }
     }
     
-    private TableViewFinder findChildViews_4_11(Region region, byte[] tenantId, byte[] schemaName, byte[] tableName, boolean stopAfterFirst) throws IOException {
-        Scan scan = new Scan();
-        byte[] startRow = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
-        byte[] stopRow = ByteUtil.nextKey(startRow);
-        scan.setStartRow(startRow);
-        scan.setStopRow(stopRow);
-        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, CHILD_TABLE_BYTES);
-        Filter filter = linkFilter;
-        linkFilter.setFilterIfMissing(true);
-        if (stopAfterFirst) {
-            filter = new FilterList(linkFilter, new PageFilter(1));
-        }
-        scan.setFilter(filter);
-        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
-        scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
-        
-        // Original region-only scanner modified due to PHOENIX-1208
-        // RegionScanner scanner = region.getScanner(scan);
-        // The following *should* work, but doesn't due to HBASE-11837
-        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
-        // HTableInterface hTable = env.getTable(systemCatalogTableName);
-        // These deprecated calls work around the issue
-        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
-            region.getTableDesc().getTableName().getName())) {
-            boolean allViewsInCurrentRegion = true;
-            int numOfChildViews = 0;
-            List<ViewInfo> viewInfoList = Lists.newArrayList();
-            try (ResultScanner scanner = hTable.getScanner(scan)) {
-                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
-                    numOfChildViews++;
-                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-                    ResultTuple resultTuple = new ResultTuple(result);
-                    resultTuple.getKey(ptr);
-                    byte[] key = ptr.copyBytes();
-                    if (checkTableKeyInRegion(key, region) != null) {
-                        allViewsInCurrentRegion = false;
-                    }
-                    byte[][] rowViewKeyMetaData = new byte[5][];
-                    getVarChars(result.getRow(), 5, rowViewKeyMetaData);
-                    byte[] viewTenantId = rowViewKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
-                    byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
-                    byte[] viewName = SchemaUtil.getTableNameFromFullName(rowViewKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
-                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
-                }
-                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
-                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
-                    tableViewFinderResult.setAllViewsNotInSingleRegion();
-                }
-                return tableViewFinderResult;
+    private void findAncestorViews(byte[] tenantId, byte[] schemaName, byte[] tableName,
+            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
+        try (Table hTable =
+                env.getTable(SchemaUtil.getPhysicalTableName(
+                    PhoenixDatabaseMetaData.SYSTEM_CATALOG

<TRUNCATED>

[12/50] [abbrv] phoenix git commit: PHOENIX-4818 Fix RAT check for missing licenses

Posted by ja...@apache.org.
PHOENIX-4818 Fix RAT check for missing licenses


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

Branch: refs/heads/omid2
Commit: 26d2460bd690954ddaf80ec5ff741b6bde050eac
Parents: 93fdd5b
Author: Vincent Poon <vi...@apache.org>
Authored: Sun Jul 22 14:19:15 2018 -0700
Committer: Vincent Poon <vi...@apache.org>
Committed: Sun Jul 22 14:19:49 2018 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/SplitSystemCatalogTests.java   | 17 +++++++++++++++++
 .../end2end/StatsEnabledSplitSystemCatalogIT.java  | 17 +++++++++++++++++
 .../apache/phoenix/query/ConnectionCachingIT.java  | 17 +++++++++++++++++
 .../schema/ParentTableNotFoundException.java       | 17 +++++++++++++++++
 4 files changed, 68 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d2460b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
index 27fc5c6..67d3fd3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
@@ -1,3 +1,20 @@
+/*
+ * 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 java.lang.annotation.ElementType;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d2460b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
index e25415a..197263f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsEnabledSplitSystemCatalogIT.java
@@ -1,3 +1,20 @@
+/*
+ * 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.util.TestUtil.analyzeTable;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d2460b/phoenix-core/src/it/java/org/apache/phoenix/query/ConnectionCachingIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/query/ConnectionCachingIT.java b/phoenix-core/src/it/java/org/apache/phoenix/query/ConnectionCachingIT.java
index b2ef052..d1dda04 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/query/ConnectionCachingIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/query/ConnectionCachingIT.java
@@ -1,3 +1,20 @@
+/*
+ * 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.query;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/26d2460b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
index 27446e0..2b66559 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
@@ -1,3 +1,20 @@
+/*
+ * 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.schema;
 
 import org.apache.phoenix.coprocessor.TableInfo;


[19/50] [abbrv] phoenix git commit: PHOENIX-4822 Fixed Spelling.

Posted by ja...@apache.org.
PHOENIX-4822 Fixed Spelling.

Closes #318

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: d4ee42907735c38f031718044bbccc5a7400c3c3
Parents: 88bf64d
Author: Jimmy Casey <ca...@gmail.com>
Authored: Sun Jul 29 21:43:55 2018 +0000
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 1 11:36:45 2018 -0400

----------------------------------------------------------------------
 phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d4ee4290/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
index dab03e7..a7a2180 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/log/LogWriter.java
@@ -21,7 +21,7 @@ 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
+ * Used by the event handler to write RingBufferEvent, this is done in a separate thread from the application configured
  * during disruptor
  */
 public interface LogWriter {


[20/50] [abbrv] phoenix git commit: PHOENIX-4751 Implement client-side hash aggregation

Posted by ja...@apache.org.
PHOENIX-4751 Implement client-side hash aggregation


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

Branch: refs/heads/omid2
Commit: 17773a8bece75f73fcf5cbc2af85e479a147e17c
Parents: d4ee429
Author: Gerald Sangudi <gs...@23andme.com>
Authored: Thu Jun 14 12:49:30 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Aug 2 20:29:42 2018 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/ClientHashAggregateIT.java  | 208 ++++++++++++++++++
 .../phoenix/execute/ClientAggregatePlan.java    |  40 +++-
 .../ClientHashAggregatingResultIterator.java    | 210 +++++++++++++++++++
 .../java/org/apache/phoenix/parse/HintNode.java |   5 +
 4 files changed, 453 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/17773a8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientHashAggregateIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientHashAggregateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientHashAggregateIT.java
new file mode 100644
index 0000000..bdc638b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientHashAggregateIT.java
@@ -0,0 +1,208 @@
+/*
+ * 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.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.junit.Test;
+
+public class ClientHashAggregateIT extends ParallelStatsDisabledIT {
+    
+    @Test
+    public void testSalted() throws Exception { 
+
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+   
+        try {
+            String table = createSalted(conn);
+            testTable(conn, table);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testUnsalted() throws Exception { 
+
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+   
+        try {
+            String table = createUnsalted(conn);
+            testTable(conn, table);
+        } finally {
+            conn.close();
+        }
+    }
+
+    private void testTable(Connection conn, String table) throws Exception {
+        verifyExplain(conn, table, false, false);
+        verifyExplain(conn, table, false, true);
+        verifyExplain(conn, table, true, false);
+        verifyExplain(conn, table, true, true);
+
+        verifyResults(conn, table, 13, 0, false, false);
+        verifyResults(conn, table, 13, 0, false, true);
+        verifyResults(conn, table, 13, 0, true, false);
+        verifyResults(conn, table, 13, 0, true, true);
+
+        verifyResults(conn, table, 13, 17, false, true);
+        verifyResults(conn, table, 13, 17, true, true);
+
+        dropTable(conn, table);
+    }
+
+    private String createSalted(Connection conn) throws Exception {
+    
+        String table = "SALTED_" + generateUniqueName();
+        String create = "CREATE TABLE " + table + " ("
+            + " keyA BIGINT NOT NULL,"
+            + " keyB BIGINT NOT NULL,"
+            + " val SMALLINT,"
+            + " CONSTRAINT pk PRIMARY KEY (keyA, keyB)"
+            + ") SALT_BUCKETS = 4";
+
+        conn.createStatement().execute(create);
+        return table;
+    }
+
+    private String createUnsalted(Connection conn) throws Exception {
+    
+        String table = "UNSALTED_" + generateUniqueName();
+        String create = "CREATE TABLE " + table + " ("
+            + " keyA BIGINT NOT NULL,"
+            + " keyB BIGINT NOT NULL,"
+            + " val SMALLINT,"
+            + " CONSTRAINT pk PRIMARY KEY (keyA, keyB)"
+            + ")";
+
+        conn.createStatement().execute(create);
+        return table;
+    }
+
+    private String getQuery(String table, boolean hash, boolean swap, boolean sort) {
+
+        String query = "SELECT /*+ USE_SORT_MERGE_JOIN"
+            + (hash ? " HASH_AGGREGATE" : "") + " */"
+            + " t1.val v1, t2.val v2, COUNT(*) c"
+            + " FROM " + table + " t1 JOIN " + table + " t2"
+            + " ON (t1.keyB = t2.keyB)"
+            + " WHERE t1.keyA = 10 AND t2.keyA = 20"
+            + " GROUP BY "
+            + (swap ? "t2.val, t1.val" : "t1.val, t2.val")
+            + (sort ? " ORDER BY t1.val, t2.val" : "")
+            ;
+
+        return query;
+    }
+
+    private void verifyExplain(Connection conn, String table, boolean swap, boolean sort) throws Exception {
+
+        String query = "EXPLAIN " + getQuery(table, true, swap, sort);
+        Statement stmt = conn.createStatement();
+        ResultSet rs = stmt.executeQuery(query);
+        String plan = QueryUtil.getExplainPlan(rs);
+        rs.close();
+        assertTrue(plan != null && plan.contains("CLIENT HASH AGGREGATE"));
+        assertTrue(plan != null && (sort == plan.contains("CLIENT SORTED BY")));
+    }
+
+    private void verifyResults(Connection conn, String table, int c1, int c2, boolean swap, boolean sort) throws Exception {
+
+        String upsert = "UPSERT INTO " + table + "(keyA, keyB, val) VALUES(?, ?, ?)";
+        PreparedStatement upsertStmt = conn.prepareStatement(upsert);
+        for (int i = 0; i < c1; i++) {
+            upsertStmt.setInt(1, 10);
+            upsertStmt.setInt(2, 100+i);
+            upsertStmt.setInt(3, 1);
+            upsertStmt.execute();
+
+            upsertStmt.setInt(1, 20);
+            upsertStmt.setInt(2, 100+i);
+            upsertStmt.setInt(3, 2);
+            upsertStmt.execute();
+        }
+        for (int i = 0; i < c2; i++) {
+            upsertStmt.setInt(1, 10);
+            upsertStmt.setInt(2, 200+i);
+            upsertStmt.setInt(3, 2);
+            upsertStmt.execute();
+
+            upsertStmt.setInt(1, 20);
+            upsertStmt.setInt(2, 200+i);
+            upsertStmt.setInt(3, 1);
+            upsertStmt.execute();
+        }
+        conn.commit();
+
+        String hashQuery = getQuery(table, true, swap, sort);
+        String sortQuery = getQuery(table, false, swap, sort);
+        Statement stmt = conn.createStatement();
+        ResultSet hrs = stmt.executeQuery(hashQuery);
+        ResultSet srs = stmt.executeQuery(sortQuery);
+
+        try {
+            if (c1 > 0) {
+                assertTrue(hrs.next());
+                assertTrue(srs.next());
+                assertEquals(hrs.getInt("v1"), srs.getInt("v1"));
+                assertEquals(hrs.getInt("v2"), srs.getInt("v2"));
+                assertEquals(hrs.getInt("c"), srs.getInt("c"));
+                assertEquals(hrs.getInt("v1"), 1);
+                assertEquals(hrs.getInt("v2"), 2);
+                assertEquals(hrs.getInt("c"), c1);
+            }
+            if (c2 > 0) {
+                assertTrue(hrs.next());
+                assertTrue(srs.next());
+                assertEquals(hrs.getInt("v1"), srs.getInt("v1"));
+                assertEquals(hrs.getInt("v2"), srs.getInt("v2"));
+                assertEquals(hrs.getInt("c"), srs.getInt("c"));
+                assertEquals(hrs.getInt("v1"), 2);
+                assertEquals(hrs.getInt("v2"), 1);
+                assertEquals(hrs.getInt("c"), c2);
+            }
+            assertFalse(hrs.next());
+            assertFalse(srs.next());
+        } finally {
+            hrs.close();
+            srs.close();
+        }
+    }
+
+    private void dropTable(Connection conn, String table) throws Exception {
+
+        String drop = "DROP TABLE " + table;
+        Statement stmt = conn.createStatement();
+        stmt.execute(drop);
+        stmt.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/17773a8b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
index c306aca..676e8ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
@@ -46,6 +46,7 @@ import org.apache.phoenix.expression.aggregator.ClientAggregators;
 import org.apache.phoenix.expression.aggregator.ServerAggregators;
 import org.apache.phoenix.iterate.AggregatingResultIterator;
 import org.apache.phoenix.iterate.BaseGroupedAggregatingResultIterator;
+import org.apache.phoenix.iterate.ClientHashAggregatingResultIterator;
 import org.apache.phoenix.iterate.DistinctAggregatingResultIterator;
 import org.apache.phoenix.iterate.FilterAggregatingResultIterator;
 import org.apache.phoenix.iterate.FilterResultIterator;
@@ -62,6 +63,7 @@ import org.apache.phoenix.iterate.SequenceResultIterator;
 import org.apache.phoenix.iterate.UngroupedAggregatingResultIterator;
 import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.TableRef;
@@ -77,6 +79,7 @@ public class ClientAggregatePlan extends ClientProcessingPlan {
     private final Expression having;
     private final ServerAggregators serverAggregators;
     private final ClientAggregators clientAggregators;
+    private final boolean useHashAgg;
     
     public ClientAggregatePlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector,
             Integer limit, Integer offset, Expression where, OrderBy orderBy, GroupBy groupBy, Expression having, QueryPlan delegate) {
@@ -90,6 +93,10 @@ public class ClientAggregatePlan extends ClientProcessingPlan {
         // another one.
         this.serverAggregators = ServerAggregators.deserialize(context.getScan()
                         .getAttribute(BaseScannerRegionObserver.AGGREGATORS), context.getConnection().getQueryServices().getConfiguration(), null);
+
+        // Extract hash aggregate hint, if any.
+        HintNode hints = statement.getHint();
+        useHashAgg = hints != null && hints.hasHint(HintNode.Hint.HASH_AGGREGATE);
     }
 
     @Override
@@ -135,17 +142,25 @@ public class ClientAggregatePlan extends ClientProcessingPlan {
             aggResultIterator = new ClientUngroupedAggregatingResultIterator(LookAheadResultIterator.wrap(iterator), serverAggregators);
             aggResultIterator = new UngroupedAggregatingResultIterator(LookAheadResultIterator.wrap(aggResultIterator), clientAggregators);
         } else {
-            if (!groupBy.isOrderPreserving()) {
-                int thresholdBytes = context.getConnection().getQueryServices().getProps().getInt(
-                        QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_SPOOL_THRESHOLD_BYTES);
-                List<Expression> keyExpressions = groupBy.getKeyExpressions();
+            List<Expression> keyExpressions = groupBy.getKeyExpressions();
+            if (groupBy.isOrderPreserving()) {
+                aggResultIterator = new ClientGroupedAggregatingResultIterator(LookAheadResultIterator.wrap(iterator), serverAggregators, keyExpressions);
+            } else {
+                int thresholdBytes = context.getConnection().getQueryServices().getProps().getInt
+                    (QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_SPOOL_THRESHOLD_BYTES);
                 List<OrderByExpression> keyExpressionOrderBy = Lists.newArrayListWithExpectedSize(keyExpressions.size());
                 for (Expression keyExpression : keyExpressions) {
                     keyExpressionOrderBy.add(new OrderByExpression(keyExpression, false, true));
                 }
-                iterator = new OrderedResultIterator(iterator, keyExpressionOrderBy, thresholdBytes, null, null, projector.getEstimatedRowByteSize());
+
+                if (useHashAgg) {
+                    // Pass in orderBy to apply any sort that has been optimized away
+                    aggResultIterator = new ClientHashAggregatingResultIterator(context, iterator, serverAggregators, keyExpressions, orderBy);
+                } else {
+                    iterator = new OrderedResultIterator(iterator, keyExpressionOrderBy, thresholdBytes, null, null, projector.getEstimatedRowByteSize());
+                    aggResultIterator = new ClientGroupedAggregatingResultIterator(LookAheadResultIterator.wrap(iterator), serverAggregators, keyExpressions);
+                }
             }
-            aggResultIterator = new ClientGroupedAggregatingResultIterator(LookAheadResultIterator.wrap(iterator), serverAggregators, groupBy.getKeyExpressions());
             aggResultIterator = new GroupedAggregatingResultIterator(LookAheadResultIterator.wrap(aggResultIterator), clientAggregators);
         }
 
@@ -183,13 +198,18 @@ public class ClientAggregatePlan extends ClientProcessingPlan {
         if (where != null) {
             planSteps.add("CLIENT FILTER BY " + where.toString());
         }
-        if (!groupBy.isEmpty()) {
-            if (!groupBy.isOrderPreserving()) {
+        if (groupBy.isEmpty()) {
+            planSteps.add("CLIENT AGGREGATE INTO SINGLE ROW");
+        } else if (groupBy.isOrderPreserving()) {
+            planSteps.add("CLIENT AGGREGATE INTO DISTINCT ROWS BY " + groupBy.getExpressions().toString());
+        } else if (useHashAgg) {
+            planSteps.add("CLIENT HASH AGGREGATE INTO DISTINCT ROWS BY " + groupBy.getExpressions().toString());
+            if (orderBy == OrderBy.FWD_ROW_KEY_ORDER_BY || orderBy == OrderBy.REV_ROW_KEY_ORDER_BY) {
                 planSteps.add("CLIENT SORTED BY " + groupBy.getKeyExpressions().toString());
             }
-            planSteps.add("CLIENT AGGREGATE INTO DISTINCT ROWS BY " + groupBy.getExpressions().toString());
         } else {
-            planSteps.add("CLIENT AGGREGATE INTO SINGLE ROW");            
+            planSteps.add("CLIENT SORTED BY " + groupBy.getKeyExpressions().toString());
+            planSteps.add("CLIENT AGGREGATE INTO DISTINCT ROWS BY " + groupBy.getExpressions().toString());
         }
         if (having != null) {
             planSteps.add("CLIENT AFTER-AGGREGATION FILTER BY " + having.toString());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/17773a8b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ClientHashAggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ClientHashAggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ClientHashAggregatingResultIterator.java
new file mode 100644
index 0000000..a07ea16
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ClientHashAggregatingResultIterator.java
@@ -0,0 +1,210 @@
+/*
+ * 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.iterate;
+
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.Aggregators;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.SizedUtil;
+import org.apache.phoenix.util.TupleUtil;
+
+/**
+ * 
+ * This class implements client-side hash aggregation in memory.
+ * Issue https://issues.apache.org/jira/browse/PHOENIX-4751.
+ * 
+ */
+public class ClientHashAggregatingResultIterator
+    implements AggregatingResultIterator {
+
+    private static final int HASH_AGG_INIT_SIZE = 64*1024;
+    private static final int CLIENT_HASH_AGG_MEMORY_CHUNK_SIZE = 64*1024;
+    private static final byte[] UNITIALIZED_KEY_BUFFER = new byte[0];
+    private final ResultIterator resultIterator;
+    private final Aggregators aggregators;
+    private final List<Expression> groupByExpressions;
+    private final OrderBy orderBy;
+    private final MemoryChunk memoryChunk;
+    private HashMap<ImmutableBytesWritable, Aggregator[]> hash;
+    private List<ImmutableBytesWritable> keyList;
+    private Iterator<ImmutableBytesWritable> keyIterator;
+
+    public ClientHashAggregatingResultIterator(StatementContext context, ResultIterator resultIterator,
+                                               Aggregators aggregators, List<Expression> groupByExpressions, OrderBy orderBy) {
+
+        Objects.requireNonNull(resultIterator);
+        Objects.requireNonNull(aggregators);
+        Objects.requireNonNull(groupByExpressions);
+        this.resultIterator = resultIterator;
+        this.aggregators = aggregators;
+        this.groupByExpressions = groupByExpressions;
+        this.orderBy = orderBy;
+        memoryChunk = context.getConnection().getQueryServices().getMemoryManager().allocate(CLIENT_HASH_AGG_MEMORY_CHUNK_SIZE);
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        if (keyIterator == null) {
+            hash = populateHash();
+            /********
+             *
+             * Perform a post-aggregation sort only when required. There are 3 possible scenarios:
+             * (1) The query DOES NOT have an ORDER BY -- in this case, we DO NOT perform a sort, and the results will be in random order.
+             * (2) The query DOES have an ORDER BY, the ORDER BY keys match the GROUP BY keys, and all the ORDER BY keys are ASCENDING
+             *     -- in this case, we DO perform a sort. THE ORDER BY has been optimized away, because the non-hash client aggregation
+             *        generates results in ascending order of the GROUP BY keys.
+             * (3) The query DOES have an ORDER BY, but the ORDER BY keys do not match the GROUP BY keys, or at least one ORDER BY key is DESCENDING
+             *     -- in this case, we DO NOT perform a sort, because the ORDER BY has not been optimized away and will be performed later by the
+             *        client aggregation code.
+             *
+             * Finally, we also handle optimization of reverse sort here. This is currently defensive, because reverse sort is not optimized away.
+             *
+             ********/
+            if (orderBy == OrderBy.FWD_ROW_KEY_ORDER_BY || orderBy == OrderBy.REV_ROW_KEY_ORDER_BY) {
+                keyList = sortKeys();
+                keyIterator = keyList.iterator();
+            } else {
+                keyIterator = hash.keySet().iterator();
+            }
+        }
+
+        if (!keyIterator.hasNext()) {
+            return null;
+        }
+
+        ImmutableBytesWritable key = keyIterator.next();
+        Aggregator[] rowAggregators = hash.get(key);
+        byte[] value = aggregators.toBytes(rowAggregators);
+        Tuple tuple = wrapKeyValueAsResult(KeyValueUtil.newKeyValue(key, SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length));
+        return tuple;
+    }
+
+    @Override
+    public void close() throws SQLException {
+        keyIterator = null;
+        keyList = null;
+        hash = null;
+        try {
+            memoryChunk.close();
+        } finally {
+            resultIterator.close();
+        }
+    }
+
+    @Override
+    public Aggregator[] aggregate(Tuple result) {
+        Aggregator[] rowAggregators = aggregators.getAggregators();
+        aggregators.reset(rowAggregators);
+        aggregators.aggregate(rowAggregators, result);
+        return rowAggregators;
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        resultIterator.explain(planSteps);
+    }
+
+    @Override
+        public String toString() {
+        return "ClientHashAggregatingResultIterator [resultIterator="
+            + resultIterator + ", aggregators=" + aggregators + ", groupByExpressions="
+            + groupByExpressions + "]";
+    }
+
+    // Copied from ClientGroupedAggregatingResultIterator
+    protected ImmutableBytesWritable getGroupingKey(Tuple tuple, ImmutableBytesWritable ptr) throws SQLException {
+        try {
+            ImmutableBytesWritable key = TupleUtil.getConcatenatedValue(tuple, groupByExpressions);
+            ptr.set(key.get(), key.getOffset(), key.getLength());
+            return ptr;
+        } catch (IOException e) {
+            throw new SQLException(e);
+        }
+    }
+
+    // Copied from ClientGroupedAggregatingResultIterator
+    protected Tuple wrapKeyValueAsResult(KeyValue keyValue) {
+        return new MultiKeyValueTuple(Collections.<Cell> singletonList(keyValue));
+    }
+
+    private HashMap<ImmutableBytesWritable, Aggregator[]> populateHash() throws SQLException {
+
+        hash = new HashMap<ImmutableBytesWritable, Aggregator[]>(HASH_AGG_INIT_SIZE, 0.75f);
+        final int aggSize = aggregators.getEstimatedByteSize();
+        long keySize = 0;
+
+        for (Tuple result = resultIterator.next(); result != null; result = resultIterator.next()) {
+            ImmutableBytesWritable key = new ImmutableBytesWritable(UNITIALIZED_KEY_BUFFER);
+            key = getGroupingKey(result, key);
+            Aggregator[] rowAggregators = hash.get(key);
+            if (rowAggregators == null) {
+                keySize += key.getSize();
+                long hashSize = SizedUtil.sizeOfMap(hash.size() + 1, SizedUtil.IMMUTABLE_BYTES_WRITABLE_SIZE, aggSize) + keySize;
+                if (hashSize > memoryChunk.getSize() + CLIENT_HASH_AGG_MEMORY_CHUNK_SIZE) {
+                    // This will throw InsufficientMemoryException if necessary
+                    memoryChunk.resize(hashSize + CLIENT_HASH_AGG_MEMORY_CHUNK_SIZE);
+                }
+
+                rowAggregators = aggregators.newAggregators();
+                hash.put(key, rowAggregators);
+            }
+
+            aggregators.aggregate(rowAggregators, result);
+        }
+
+        return hash;
+    }
+
+    private List<ImmutableBytesWritable> sortKeys() {
+        // This will throw InsufficientMemoryException if necessary
+        memoryChunk.resize(memoryChunk.getSize() + SizedUtil.sizeOfArrayList(hash.size()));
+
+        keyList = new ArrayList<ImmutableBytesWritable>(hash.size());
+        keyList.addAll(hash.keySet());
+        Comparator<ImmutableBytesWritable> comp = new ImmutableBytesWritable.Comparator();
+        if (orderBy == OrderBy.REV_ROW_KEY_ORDER_BY) {
+            comp = Collections.reverseOrder(comp);
+        }
+        Collections.sort(keyList, comp);
+        return keyList;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/17773a8b/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 39e9b05..02a44ad 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
@@ -108,6 +108,11 @@ public class HintNode {
          * Enforces a forward scan.
          */
         FORWARD_SCAN,
+        /**
+         * Prefer a hash aggregate over a sort plus streaming aggregate.
+         * Issue https://issues.apache.org/jira/browse/PHOENIX-4751.
+         */
+        HASH_AGGREGATE,
     };
 
     private final Map<Hint,String> hints;


[44/50] [abbrv] phoenix git commit: PHOENIX-4666 Persistent subquery cache for hash joins

Posted by ja...@apache.org.
PHOENIX-4666 Persistent subquery cache for hash joins

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: 012bb3182aff545ca11def3b1d79955f21636b2f
Parents: 1145eda
Author: Marcell Ortutay <ma...@gmail.com>
Authored: Thu Mar 29 12:59:03 2018 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Mon Sep 10 21:29:51 2018 -0400

----------------------------------------------------------------------
 .../end2end/join/HashJoinPersistentCacheIT.java | 167 +++++++++++++++
 .../org/apache/phoenix/cache/GlobalCache.java   |  22 +-
 .../apache/phoenix/cache/ServerCacheClient.java |  59 ++++--
 .../org/apache/phoenix/cache/TenantCache.java   |   2 +-
 .../apache/phoenix/cache/TenantCacheImpl.java   | 209 ++++++++++++++++---
 .../apache/phoenix/compile/QueryCompiler.java   |   9 +-
 .../phoenix/compile/StatementContext.java       |  21 +-
 .../coprocessor/HashJoinRegionScanner.java      |   4 +-
 .../coprocessor/ServerCachingEndpointImpl.java  |   2 +-
 .../generated/ServerCachingProtos.java          | 117 +++++++++--
 .../apache/phoenix/execute/HashJoinPlan.java    | 104 +++++++--
 .../phoenix/iterate/BaseResultIterators.java    |   8 +-
 .../phoenix/iterate/TableResultIterator.java    |   6 +-
 .../apache/phoenix/join/HashCacheClient.java    |  24 ++-
 .../apache/phoenix/join/HashCacheFactory.java   |  11 +
 .../java/org/apache/phoenix/parse/HintNode.java |   4 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java     |   1 +
 .../apache/phoenix/cache/TenantCacheTest.java   | 112 ++++++++--
 .../src/main/ServerCachingService.proto         |   1 +
 phoenix-protocol/src/main/build-proto.sh        |   6 +
 21 files changed, 773 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinPersistentCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinPersistentCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinPersistentCacheIT.java
new file mode 100644
index 0000000..2f072b8
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinPersistentCacheIT.java
@@ -0,0 +1,167 @@
+/*
+ * 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.join;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.apache.phoenix.end2end.join.HashJoinCacheIT.InvalidateHashCache;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Test;
+
+public class HashJoinPersistentCacheIT extends BaseJoinIT {
+
+    @Override
+    protected String getTableName(Connection conn, String virtualName) throws Exception {
+        String realName = super.getTableName(conn, virtualName);
+        TestUtil.addCoprocessor(conn, SchemaUtil.normalizeFullTableName(realName),
+                InvalidateHashCache.class);
+        return realName;
+    }
+
+    @Test
+    public void testPersistentCache() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        createTestTable(getUrl(),
+                "CREATE TABLE IF NOT EXISTS states (state CHAR(2) " +
+                "NOT NULL, name VARCHAR NOT NULL CONSTRAINT my_pk PRIMARY KEY (state, name))");
+        createTestTable(getUrl(),
+                "CREATE TABLE IF NOT EXISTS cities (state CHAR(2) " +
+                 "NOT NULL, city VARCHAR NOT NULL, population BIGINT " +
+                  "CONSTRAINT my_pk PRIMARY KEY (state, city))");
+
+        conn.prepareStatement(
+                "UPSERT INTO states VALUES ('CA', 'California')").executeUpdate();
+        conn.prepareStatement(
+                "UPSERT INTO states VALUES ('AZ', 'Arizona')").executeUpdate();
+        conn.prepareStatement(
+                "UPSERT INTO cities VALUES ('CA', 'San Francisco', 50000)").executeUpdate();
+        conn.prepareStatement(
+                "UPSERT INTO cities VALUES ('CA', 'Sacramento', 3000)").executeUpdate();
+        conn.prepareStatement(
+                "UPSERT INTO cities VALUES ('AZ', 'Phoenix', 20000)").executeUpdate();
+        conn.commit();
+
+        /* First, run query without using the persistent cache. This should return
+        * different results after an UPSERT takes place.
+        */
+        ResultSet rs = conn.prepareStatement(
+                "SELECT SUM(population) FROM states s "
+                +"JOIN cities c ON c.state = s.state").executeQuery();
+        rs.next();
+        int population1 = rs.getInt(1);
+
+        conn.prepareStatement("UPSERT INTO cities VALUES ('CA', 'Mt View', 1500)").executeUpdate();
+        conn.commit();
+        rs = conn.prepareStatement(
+                "SELECT SUM(population) FROM states s " +
+                 "JOIN cities c ON c.state = s.state").executeQuery();
+        rs.next();
+        int population2 = rs.getInt(1);
+
+        assertEquals(73000, population1);
+        assertEquals(74500, population2);
+
+        /* Second, run query using the persistent cache. This should return the
+        * same results after an UPSERT takes place.
+        */
+        rs = conn.prepareStatement(
+                "SELECT /*+ USE_PERSISTENT_CACHE */ SUM(population) FROM states s " +
+                 "JOIN cities c ON c.state = s.state").executeQuery();
+        rs.next();
+        int population3 = rs.getInt(1);
+
+        conn.prepareStatement(
+                "UPSERT INTO cities VALUES ('CA', 'Palo Alto', 2000)").executeUpdate();
+        conn.commit();
+
+        rs = conn.prepareStatement(
+                "SELECT /*+ USE_PERSISTENT_CACHE */ SUM(population) " +
+                 "FROM states s JOIN cities c ON c.state = s.state").executeQuery();
+        rs.next();
+        int population4 = rs.getInt(1);
+        rs = conn.prepareStatement(
+                "SELECT SUM(population) FROM states s JOIN cities c ON c.state = s.state")
+                .executeQuery();
+        rs.next();
+        int population5 = rs.getInt(1);
+
+        assertEquals(74500, population3);
+        assertEquals(74500, population4);
+        assertEquals(76500, population5);
+
+        /* Let's make sure caches can be used across queries. We'll set up a
+        * cache, and make sure it is used on two different queries with the
+        * same subquery.
+        */
+
+        String sumQueryCached = "SELECT /*+ USE_PERSISTENT_CACHE */ SUM(population) " +
+                "FROM cities c JOIN (SELECT state FROM states WHERE state LIKE 'C%') sq " +
+                "ON sq.state = c.state";
+        String distinctQueryCached = "SELECT /*+ USE_PERSISTENT_CACHE */ " +
+                "COUNT(DISTINCT(c.city)) FROM cities c " +
+                "JOIN (SELECT state FROM states WHERE state LIKE 'C%') sq " +
+                "ON sq.state = c.state";
+        String sumQueryUncached = sumQueryCached.replace(
+                "/*+ USE_PERSISTENT_CACHE */", "");
+        String distinctQueryUncached = distinctQueryCached.replace(
+                "/*+ USE_PERSISTENT_CACHE */", "");
+
+        rs = conn.prepareStatement(sumQueryCached).executeQuery();
+        rs.next();
+        int population6 = rs.getInt(1);
+        rs = conn.prepareStatement(distinctQueryCached).executeQuery();
+        rs.next();
+        int distinct1 = rs.getInt(1);
+        assertEquals(4, distinct1);
+
+        // Add a new city that matches the queries. This should not affect results
+        // using persistent caching.
+        conn.prepareStatement("UPSERT INTO states VALUES ('CO', 'Colorado')").executeUpdate();
+        conn.prepareStatement("UPSERT INTO cities VALUES ('CO', 'Denver', 6000)").executeUpdate();
+        conn.commit();
+
+        rs = conn.prepareStatement(sumQueryCached).executeQuery();
+        rs.next();
+        int population7 = rs.getInt(1);
+        assertEquals(population6, population7);
+        rs = conn.prepareStatement(distinctQueryCached).executeQuery();
+        rs.next();
+        int distinct2 = rs.getInt(1);
+        assertEquals(distinct1, distinct2);
+
+        // Finally, make sure uncached queries give up to date results
+        rs = conn.prepareStatement(sumQueryUncached).executeQuery();
+        rs.next();
+        int population8 = rs.getInt(1);
+        assertEquals(population8, 62500);
+        rs = conn.prepareStatement(distinctQueryUncached).executeQuery();
+        rs.next();
+        int distinct3 = rs.getInt(1);
+        assertEquals(5, distinct3);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
index ae77174..5f3e29b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
@@ -148,7 +148,12 @@ public class GlobalCache extends TenantCacheImpl {
     
     private GlobalCache(Configuration config) {
         super(new GlobalMemoryManager(getMaxMemorySize(config)),
-              config.getInt(QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB, QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS));
+              config.getInt(
+                      QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB,
+                      QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS),
+              config.getInt(
+                      QueryServices.MAX_SERVER_CACHE_PERSISTENCE_TIME_TO_LIVE_MS_ATTRIB,
+                      QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_PERSISTENCE_TIME_TO_LIVE_MS));
         this.config = config;
     }
     
@@ -164,9 +169,18 @@ public class GlobalCache extends TenantCacheImpl {
     public TenantCache getChildTenantCache(ImmutableBytesPtr tenantId) {
         TenantCache tenantCache = perTenantCacheMap.get(tenantId);
         if (tenantCache == null) {
-            int maxTenantMemoryPerc = config.getInt(MAX_TENANT_MEMORY_PERC_ATTRIB, QueryServicesOptions.DEFAULT_MAX_TENANT_MEMORY_PERC);
-            int maxServerCacheTimeToLive = config.getInt(QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB, QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS);
-            TenantCacheImpl newTenantCache = new TenantCacheImpl(new ChildMemoryManager(getMemoryManager(), maxTenantMemoryPerc), maxServerCacheTimeToLive);
+            int maxTenantMemoryPerc = config.getInt(
+                    MAX_TENANT_MEMORY_PERC_ATTRIB,
+                    QueryServicesOptions.DEFAULT_MAX_TENANT_MEMORY_PERC);
+            int maxServerCacheTimeToLive = config.getInt(
+                    QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB,
+                    QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS);
+            int maxServerCachePersistenceTimeToLive = config.getInt(
+                    QueryServices.MAX_SERVER_CACHE_PERSISTENCE_TIME_TO_LIVE_MS_ATTRIB,
+                    QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_PERSISTENCE_TIME_TO_LIVE_MS);
+            TenantCacheImpl newTenantCache = new TenantCacheImpl(
+                    new ChildMemoryManager(getMemoryManager(), maxTenantMemoryPerc),
+                    maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive);
             tenantCache = perTenantCacheMap.putIfAbsent(tenantId, newTenantCache);
             if (tenantCache == null) {
                 tenantCache = newTenantCache;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 5e284bd..011a6f8 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
@@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
@@ -59,6 +61,7 @@ import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.AddServerCac
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.RemoveServerCacheRequest;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.RemoveServerCacheResponse;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ServerCachingService;
+import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.job.JobManager.JobCallable;
 import org.apache.phoenix.join.HashCacheFactory;
@@ -75,6 +78,8 @@ import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.ScanUtil;
 
+import com.google.protobuf.ByteString;
+
 /**
  * 
  * Client for sending cache to each region server
@@ -215,22 +220,46 @@ public class ServerCacheClient {
                 }
             }
         }
-        
     }
-    
-    public ServerCache addServerCache(ScanRanges keyRanges, final ImmutableBytesWritable cachePtr, final byte[] txState,
-            final ServerCacheFactory cacheFactory, final PTable cacheUsingTable) throws SQLException {
-        return addServerCache(keyRanges, cachePtr, txState, cacheFactory, cacheUsingTable, false);
+
+    public ServerCache createServerCache(byte[] cacheId, QueryPlan delegate)
+            throws SQLException, IOException {
+        PTable cacheUsingTable = delegate.getTableRef().getTable();
+        ConnectionQueryServices services = delegate.getContext().getConnection().getQueryServices();
+        List<HRegionLocation> locations = services.getAllTableRegions(
+                cacheUsingTable.getPhysicalName().getBytes());
+        int nRegions = locations.size();
+        Set<HRegionLocation> servers = new HashSet<>(nRegions);
+        cacheUsingTableMap.put(Bytes.mapKey(cacheId), cacheUsingTable);
+        return new ServerCache(cacheId, servers, new ImmutableBytesWritable(
+                new byte[]{}), services, false);
     }
-    
-    public ServerCache addServerCache(ScanRanges keyRanges, final ImmutableBytesWritable cachePtr, final byte[] txState,
-            final ServerCacheFactory cacheFactory, final PTable cacheUsingTable, boolean storeCacheOnClient)
+
+    public ServerCache addServerCache(
+            ScanRanges keyRanges, final ImmutableBytesWritable cachePtr, final byte[] txState,
+            final ServerCacheFactory cacheFactory, final PTable cacheUsingTable)
             throws SQLException {
+        return addServerCache(keyRanges, cachePtr, txState, cacheFactory, cacheUsingTable, false);
+    }
+
+    public ServerCache addServerCache(
+            ScanRanges keyRanges, final ImmutableBytesWritable cachePtr, final byte[] txState,
+            final ServerCacheFactory cacheFactory, final PTable cacheUsingTable,
+            boolean storeCacheOnClient) throws SQLException {
+        final byte[] cacheId = ServerCacheClient.generateId();
+        return addServerCache(keyRanges, cacheId, cachePtr, txState, cacheFactory,
+                cacheUsingTable, false, storeCacheOnClient);
+    }
+
+    public ServerCache addServerCache(
+            ScanRanges keyRanges, final byte[] cacheId, final ImmutableBytesWritable cachePtr,
+            final byte[] txState, final ServerCacheFactory cacheFactory,
+            final PTable cacheUsingTable, final boolean usePersistentCache,
+            boolean storeCacheOnClient) throws SQLException {
         ConnectionQueryServices services = connection.getQueryServices();
         List<Closeable> closeables = new ArrayList<Closeable>();
         ServerCache hashCacheSpec = null;
         SQLException firstException = null;
-        final byte[] cacheId = generateId();
         /**
          * Execute EndPoint in parallel on each server to send compressed hash cache 
          */
@@ -251,7 +280,7 @@ public class ServerCacheClient {
                 byte[] regionEndKey = entry.getRegionInfo().getEndKey();
                 if ( ! servers.contains(entry) && 
                         keyRanges.intersectRegion(regionStartKey, regionEndKey,
-                                cacheUsingTable.getIndexType() == IndexType.LOCAL)) {  
+                                cacheUsingTable.getIndexType() == IndexType.LOCAL)) {
                     // Call RPC once per server
                     servers.add(entry);
                     if (LOG.isDebugEnabled()) {LOG.debug(addCustomAnnotations("Adding cache entry to be sent for " + entry, connection));}
@@ -262,7 +291,7 @@ public class ServerCacheClient {
                         
                         @Override
                         public Boolean call() throws Exception {
-                            return addServerCache(htable, key, cacheUsingTable, cacheId, cachePtr, cacheFactory, txState);
+                            return addServerCache(htable, key, cacheUsingTable, cacheId, cachePtr, cacheFactory, txState, usePersistentCache);
                         }
 
                         /**
@@ -291,7 +320,7 @@ public class ServerCacheClient {
             for (Future<Boolean> future : futures) {
                 future.get(timeoutMs, TimeUnit.MILLISECONDS);
             }
-            
+
             cacheUsingTableMap.put(Bytes.mapKey(cacheId), cacheUsingTable);
             success = true;
         } catch (SQLException e) {
@@ -444,7 +473,7 @@ public class ServerCacheClient {
             }
 			if (cache.addServer(tableRegionLocation) || services.getProps().getBoolean(HASH_JOIN_SERVER_CACHE_RESEND_PER_SERVER,false)) {
 				success = addServerCache(table, startkeyOfRegion, pTable, cacheId, cache.getCachePtr(), cacheFactory,
-						txState);
+						txState, false);
 			}
 			return success;
         } finally {
@@ -453,7 +482,7 @@ public class ServerCacheClient {
     }
     
     public boolean addServerCache(HTableInterface htable, byte[] key, final PTable cacheUsingTable, final byte[] cacheId,
-            final ImmutableBytesWritable cachePtr, final ServerCacheFactory cacheFactory, final byte[] txState)
+            final ImmutableBytesWritable cachePtr, final ServerCacheFactory cacheFactory, final byte[] txState, final boolean usePersistentCache)
             throws Exception {
         byte[] keyInRegion = getKeyInRegion(key);
         final Map<byte[], AddServerCacheResponse> results;
@@ -483,6 +512,7 @@ public class ServerCacheClient {
                                 builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
                             }
                             builder.setCacheId(ByteStringer.wrap(cacheId));
+                            builder.setUsePersistentCache(usePersistentCache);
                             builder.setCachePtr(org.apache.phoenix.protobuf.ProtobufUtil.toProto(cachePtr));
                             builder.setHasProtoBufIndexMaintainer(true);
                             ServerCacheFactoryProtos.ServerCacheFactory.Builder svrCacheFactoryBuider = ServerCacheFactoryProtos.ServerCacheFactory
@@ -501,7 +531,6 @@ public class ServerCacheClient {
         }
         if (results != null && results.size() == 1) { return results.values().iterator().next().getReturn(); }
         return false;
-
     }
     
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
index c4e82c2..e37d4d4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
@@ -36,7 +36,7 @@ import org.apache.phoenix.memory.MemoryManager;
 public interface TenantCache {
     MemoryManager getMemoryManager();
     Closeable getServerCache(ImmutableBytesPtr cacheId);
-    Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory, boolean useProtoForIndexMaintainer, int clientVersion) throws SQLException;
+    Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory, boolean useProtoForIndexMaintainer, boolean usePersistentCache, int clientVersion) throws SQLException;
     void removeServerCache(ImmutableBytesPtr cacheId);
     void removeAllServerCache();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
index 1dc59bc..dc4c9e3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
@@ -18,16 +18,22 @@
 package org.apache.phoenix.cache;
 
 import java.io.Closeable;
+import java.io.IOException;
 import java.sql.SQLException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.MemoryManager;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.util.Closeables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Ticker;
 import com.google.common.cache.Cache;
@@ -44,18 +50,86 @@ import com.google.common.cache.RemovalNotification;
  * @since 0.1
  */
 public class TenantCacheImpl implements TenantCache {
+    private static final Logger logger = LoggerFactory.getLogger(TenantCacheImpl.class);
     private final int maxTimeToLiveMs;
+    private final int maxPersistenceTimeToLiveMs;
     private final MemoryManager memoryManager;
     private final Ticker ticker;
-    private volatile Cache<ImmutableBytesPtr, Closeable> serverCaches;
 
-    public TenantCacheImpl(MemoryManager memoryManager, int maxTimeToLiveMs) {
-        this(memoryManager, maxTimeToLiveMs, Ticker.systemTicker());
+    // Two caches exist: the "serverCaches" cache which is used for handling live
+    // queries, and the "persistentServerCaches" cache which is used to store data
+    // between queries. If we are out of memory, attempt to clear out entries from
+    // the persistent cache before throwing an exception.
+    private volatile Cache<ImmutableBytesPtr, CacheEntry> serverCaches;
+    private volatile Cache<ImmutableBytesPtr, CacheEntry> persistentServerCaches;
+
+    private final long EVICTION_MARGIN_BYTES = 10000000;
+
+    private class CacheEntry implements Comparable<CacheEntry>, Closeable {
+        private ImmutableBytesPtr cacheId;
+        private ImmutableBytesWritable cachePtr;
+        private int hits;
+        private int liveQueriesCount;
+        private boolean usePersistentCache;
+        private long size;
+        private Closeable closeable;
+
+        public CacheEntry(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, 
+                ServerCacheFactory cacheFactory, byte[] txState, MemoryChunk chunk,
+                boolean usePersistentCache, boolean useProtoForIndexMaintainer,
+                int clientVersion) throws SQLException {
+            this.cacheId = cacheId;
+            this.cachePtr = cachePtr;
+            this.size = cachePtr.getLength();
+            this.hits = 0;
+            this.liveQueriesCount = 0;
+            this.usePersistentCache = usePersistentCache;
+            this.closeable = cacheFactory.newCache(cachePtr, txState, chunk, useProtoForIndexMaintainer, clientVersion);
+        }
+
+        public void close() throws IOException {
+            this.closeable.close();
+        }
+
+        synchronized public void incrementLiveQueryCount() {
+            liveQueriesCount++;
+            hits++;
+        }
+
+        synchronized public void decrementLiveQueryCount() {
+            liveQueriesCount--;
+        }
+
+        synchronized public boolean isLive() {
+            return liveQueriesCount > 0;
+        }
+
+        public boolean getUsePersistentCache() {
+            return usePersistentCache;
+        }
+
+        public ImmutableBytesPtr getCacheId() {
+            return cacheId;
+        }
+
+        private Float rank() {
+            return (float)hits;
+        }
+
+        @Override
+        public int compareTo(CacheEntry o) {
+            return rank().compareTo(o.rank());
+        }
+    }
+
+    public TenantCacheImpl(MemoryManager memoryManager, int maxTimeToLiveMs, int maxPersistenceTimeToLiveMs) {
+        this(memoryManager, maxTimeToLiveMs, maxPersistenceTimeToLiveMs, Ticker.systemTicker());
     }
     
-    public TenantCacheImpl(MemoryManager memoryManager, int maxTimeToLiveMs, Ticker ticker) {
+    public TenantCacheImpl(MemoryManager memoryManager, int maxTimeToLiveMs, int maxPersistenceTimeToLiveMs, Ticker ticker) {
         this.memoryManager = memoryManager;
         this.maxTimeToLiveMs = maxTimeToLiveMs;
+        this.maxPersistenceTimeToLiveMs = maxPersistenceTimeToLiveMs;
         this.ticker = ticker;
     }
     
@@ -69,6 +143,9 @@ public class TenantCacheImpl implements TenantCache {
             if (serverCaches != null) {
                 serverCaches.cleanUp();
             }
+            if (persistentServerCaches != null) {
+                persistentServerCaches.cleanUp();
+            }
         }
     }
     
@@ -77,57 +154,133 @@ public class TenantCacheImpl implements TenantCache {
         return memoryManager;
     }
 
-    private Cache<ImmutableBytesPtr,Closeable> getServerCaches() {
+    private Cache<ImmutableBytesPtr,CacheEntry> getServerCaches() {
         /* Delay creation of this map until it's needed */
         if (serverCaches == null) {
             synchronized(this) {
                 if (serverCaches == null) {
-                    serverCaches = CacheBuilder.newBuilder()
-                        .expireAfterAccess(maxTimeToLiveMs, TimeUnit.MILLISECONDS)
-                        .ticker(getTicker())
-                        .removalListener(new RemovalListener<ImmutableBytesPtr, Closeable>(){
-                            @Override
-                            public void onRemoval(RemovalNotification<ImmutableBytesPtr, Closeable> notification) {
-                                Closeables.closeAllQuietly(Collections.singletonList(notification.getValue()));
-                            }
-                        })
-                        .build();
+                    serverCaches = buildCache(maxTimeToLiveMs, false);
                 }
             }
         }
         return serverCaches;
     }
-    
-    @Override
+
+    private Cache<ImmutableBytesPtr,CacheEntry> getPersistentServerCaches() {
+        /* Delay creation of this map until it's needed */
+        if (persistentServerCaches == null) {
+            synchronized(this) {
+                if (persistentServerCaches == null) {
+                    persistentServerCaches = buildCache(maxPersistenceTimeToLiveMs, true);
+                }
+            }
+        }
+        return persistentServerCaches;
+    }
+
+    private Cache<ImmutableBytesPtr, CacheEntry> buildCache(final int ttl, final boolean isPersistent) {
+        CacheBuilder<Object, Object> builder = CacheBuilder.newBuilder();
+        if (isPersistent) {
+            builder.expireAfterWrite(ttl, TimeUnit.MILLISECONDS);
+        } else {
+            builder.expireAfterAccess(ttl, TimeUnit.MILLISECONDS);
+        }
+        return builder
+            .ticker(getTicker())
+            .removalListener(new RemovalListener<ImmutableBytesPtr, CacheEntry>(){
+                @Override
+                public void onRemoval(RemovalNotification<ImmutableBytesPtr, CacheEntry> notification) {
+                    if (isPersistent || !notification.getValue().getUsePersistentCache()) {
+                        Closeables.closeAllQuietly(Collections.singletonList(notification.getValue()));
+                    }
+                }
+            })
+            .build();
+    }
+
+    synchronized private void evictInactiveEntries(long bytesNeeded) {
+        logger.debug("Trying to evict inactive cache entries to free up " + bytesNeeded + " bytes");
+        CacheEntry[] entries = getPersistentServerCaches().asMap().values().toArray(new CacheEntry[]{});
+        Arrays.sort(entries);
+        long available = this.getMemoryManager().getAvailableMemory();
+        for (int i = 0; i < entries.length && available < bytesNeeded; i++) {
+            CacheEntry entry = entries[i];
+            ImmutableBytesPtr cacheId = entry.getCacheId();
+            getPersistentServerCaches().invalidate(cacheId);
+            available = this.getMemoryManager().getAvailableMemory();
+            logger.debug("Evicted cache ID " + Bytes.toLong(cacheId.get()) + ", we now have " + available + " bytes available");
+        }
+    }
+
+    private CacheEntry getIfPresent(ImmutableBytesPtr cacheId) {
+        CacheEntry entry = getPersistentServerCaches().getIfPresent(cacheId);
+        if (entry != null) {
+            return entry;
+        }
+        return getServerCaches().getIfPresent(cacheId);
+    }
+
+	@Override
     public Closeable getServerCache(ImmutableBytesPtr cacheId) {
         getServerCaches().cleanUp();
-        return getServerCaches().getIfPresent(cacheId);
+        CacheEntry entry = getIfPresent(cacheId);
+        if (entry == null) {
+            return null;
+        }
+        return entry.closeable;
     }
-    
+
     @Override
-    public Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory, boolean useProtoForIndexMaintainer, int clientVersion) throws SQLException {
+    public Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory, boolean useProtoForIndexMaintainer, boolean usePersistentCache, int clientVersion) throws SQLException {
         getServerCaches().cleanUp();
-        MemoryChunk chunk = this.getMemoryManager().allocate(cachePtr.getLength() + txState.length);
+        long available = this.getMemoryManager().getAvailableMemory();
+        int size = cachePtr.getLength() + txState.length;
+        if (size > available) {
+            evictInactiveEntries(size - available + EVICTION_MARGIN_BYTES);
+        }
+        MemoryChunk chunk = this.getMemoryManager().allocate(size);
         boolean success = false;
         try {
-            Closeable element = cacheFactory.newCache(cachePtr, txState, chunk, useProtoForIndexMaintainer, clientVersion);
-            getServerCaches().put(cacheId, element);
+            CacheEntry entry;
+            synchronized(this) {
+                entry = getIfPresent(cacheId);
+                if (entry == null) {
+                    entry = new CacheEntry(
+                            cacheId, cachePtr, cacheFactory, txState, chunk,
+                            usePersistentCache, useProtoForIndexMaintainer,
+                            clientVersion);
+                    getServerCaches().put(cacheId, entry);
+                    if (usePersistentCache) {
+                        getPersistentServerCaches().put(cacheId, entry);
+                    }
+                }
+                entry.incrementLiveQueryCount();
+            }
             success = true;
-            return element;
+            return entry;
         } finally {
             if (!success) {
                 Closeables.closeAllQuietly(Collections.singletonList(chunk));
             }
-        }           
+        }
     }
-    
+
     @Override
-    public void removeServerCache(ImmutableBytesPtr cacheId) {
-        getServerCaches().invalidate(cacheId);
+    synchronized public void removeServerCache(ImmutableBytesPtr cacheId) {
+        CacheEntry entry = getServerCaches().getIfPresent(cacheId);
+        if (entry == null) {
+            return;
+        }
+        entry.decrementLiveQueryCount();
+        if (!entry.isLive()) {
+            logger.debug("Cache ID " + Bytes.toLong(cacheId.get()) + " is no longer live, invalidate it");
+            getServerCaches().invalidate(cacheId);
+        }
     }
 
     @Override
     public void removeAllServerCache() {
         getServerCaches().invalidateAll();
+        getPersistentServerCaches().invalidateAll();
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 3e5f5ee..603da0b 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
@@ -100,6 +100,7 @@ public class QueryCompiler {
     private final SequenceManager sequenceManager;
     private final boolean projectTuples;
     private final boolean noChildParentJoinOptimization;
+    private final boolean usePersistentCache;
     private final boolean optimizeSubquery;
     private final Map<TableRef, QueryPlan> dataPlans;
     private final boolean costBased;
@@ -117,7 +118,8 @@ public class QueryCompiler {
         this.parallelIteratorFactory = parallelIteratorFactory;
         this.sequenceManager = sequenceManager;
         this.projectTuples = projectTuples;
-        this.noChildParentJoinOptimization = select.getHint().hasHint(Hint.NO_CHILD_PARENT_JOIN_OPTIMIZATION);
+        this.noChildParentJoinOptimization = select.getHint().hasHint(Hint.NO_CHILD_PARENT_JOIN_OPTIMIZATION) || select.getHint().hasHint(Hint.USE_PERSISTENT_CACHE);
+        this.usePersistentCache = select.getHint().hasHint(Hint.USE_PERSISTENT_CACHE);
         ConnectionQueryServices services = statement.getConnection().getQueryServices();
         this.costBased = services.getProps().getBoolean(QueryServices.COST_BASED_OPTIMIZER_ENABLED, QueryServicesOptions.DEFAULT_COST_BASED_OPTIMIZER_ENABLED);
         scan.setLoadColumnFamiliesOnDemand(true);
@@ -314,7 +316,7 @@ public class QueryCompiler {
                     if (i < count - 1) {
                         fieldPositions[i + 1] = fieldPositions[i] + (tables[i] == null ? 0 : (tables[i].getColumns().size() - tables[i].getPKColumns().size()));
                     }
-                    hashPlans[i] = new HashSubPlan(i, subPlans[i], optimized ? null : hashExpressions, joinSpec.isSingleValueOnly(), keyRangeLhsExpression, keyRangeRhsExpression);
+                    hashPlans[i] = new HashSubPlan(i, subPlans[i], optimized ? null : hashExpressions, joinSpec.isSingleValueOnly(), usePersistentCache, keyRangeLhsExpression, keyRangeRhsExpression);
                 }
                 TupleProjector.serializeProjectorIntoScan(context.getScan(), tupleProjector);
                 QueryPlan plan = compileSingleFlatQuery(context, query, binds, asSubquery, !asSubquery && joinTable.isAllLeftJoin(), null, !table.isSubselect() && projectPKColumns ? tupleProjector : null, true);
@@ -381,9 +383,10 @@ public class QueryCompiler {
                 HashJoinInfo joinInfo = new HashJoinInfo(projectedTable, joinIds, new List[]{joinExpressions},
                         new JoinType[]{type == JoinType.Right ? JoinType.Left : type}, new boolean[]{true},
                         new PTable[]{lhsTable}, new int[]{fieldPosition}, postJoinFilterExpression, QueryUtil.getOffsetLimit(limit, offset));
+                boolean usePersistentCache = joinTable.getStatement().getHint().hasHint(Hint.USE_PERSISTENT_CACHE);
                 Pair<Expression, Expression> keyRangeExpressions = new Pair<Expression, Expression>(null, null);
                 getKeyExpressionCombinations(keyRangeExpressions, context, joinTable.getStatement(), rhsTableRef, type, joinExpressions, hashExpressions);
-                return HashJoinPlan.create(joinTable.getStatement(), rhsPlan, joinInfo, new HashSubPlan[]{new HashSubPlan(0, lhsPlan, hashExpressions, false, keyRangeExpressions.getFirst(), keyRangeExpressions.getSecond())});
+                return HashJoinPlan.create(joinTable.getStatement(), rhsPlan, joinInfo, new HashSubPlan[]{new HashSubPlan(0, lhsPlan, hashExpressions, false, usePersistentCache, keyRangeExpressions.getFirst(), keyRangeExpressions.getSecond())});
             }
             case SORT_MERGE: {
                 JoinTable lhsJoin = joinTable.getSubJoinTableWithoutPostFilters();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 eb195c2..cc38870 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
@@ -74,6 +74,7 @@ public class StatementContext {
     private final ImmutableBytesWritable tempPtr;
     private final PhoenixStatement statement;
     private final Map<PColumn, Integer> dataColumns;
+    private Map<Long, Boolean> retryingPersistentCache;
 
     private long currentTime = QueryConstants.UNSET_TIMESTAMP;
     private ScanRanges scanRanges = ScanRanges.EVERYTHING;
@@ -138,6 +139,7 @@ public class StatementContext {
         this.subqueryResults = Maps.<SelectStatement, Object> newHashMap();
         this.readMetricsQueue = new ReadMetricQueue(isRequestMetricsEnabled,connection.getLogLevel());
         this.overAllQueryMetrics = new OverAllQueryMetrics(isRequestMetricsEnabled,connection.getLogLevel());
+        this.retryingPersistentCache = Maps.<Long, Boolean> newHashMap();
     }
 
     /**
@@ -326,5 +328,22 @@ public class StatementContext {
     public void setClientSideUpsertSelect(boolean isClientSideUpsertSelect) {
         this.isClientSideUpsertSelect = isClientSideUpsertSelect;
     }
-    
+
+    /*
+     * setRetryingPersistentCache can be used to override the USE_PERSISTENT_CACHE hint and disable the use of the
+     * persistent cache for a specific cache ID. This can be used to retry queries that failed when using the persistent
+     * cache.
+     */
+    public void setRetryingPersistentCache(long cacheId) {
+        retryingPersistentCache.put(cacheId, true);
+    }
+
+    public boolean getRetryingPersistentCache(long cacheId) {
+        Boolean retrying = retryingPersistentCache.get(cacheId);
+        if (retrying == null) {
+            return false;
+        } else {
+            return retrying;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
index d82aaba..96af154 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -51,7 +51,6 @@ import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.tuple.PositionBasedResultTuple;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
-import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.TupleUtil;
 
@@ -122,8 +121,7 @@ public class HashJoinRegionScanner implements RegionScanner {
             }
             HashCache hashCache = (HashCache)cache.getServerCache(joinId);
             if (hashCache == null) {
-                Exception cause = new HashJoinCacheNotFoundException(
-                        Bytes.toLong(ByteUtil.copyKeyBytesIfNecessary(joinId)));
+                Exception cause = new HashJoinCacheNotFoundException(Bytes.toLong(joinId.get()));
                 throw new DoNotRetryIOException(cause.getMessage(), cause);
             }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 9d78659..86219c7 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
@@ -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() : ScanUtil.UNKNOWN_CLIENT_VERSION);
+              request.getUsePersistentCache(), 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/012bb318/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
index fdca334..c42b9df 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
@@ -5760,6 +5760,16 @@ public final class ServerCachingProtos {
      * <code>optional int32 clientVersion = 7;</code>
      */
     int getClientVersion();
+
+    // optional bool usePersistentCache = 8;
+    /**
+     * <code>optional bool usePersistentCache = 8;</code>
+     */
+    boolean hasUsePersistentCache();
+    /**
+     * <code>optional bool usePersistentCache = 8;</code>
+     */
+    boolean getUsePersistentCache();
   }
   /**
    * Protobuf type {@code AddServerCacheRequest}
@@ -5863,6 +5873,11 @@ public final class ServerCachingProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 64: {
+              bitField0_ |= 0x00000080;
+              usePersistentCache_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -6027,6 +6042,22 @@ public final class ServerCachingProtos {
       return clientVersion_;
     }
 
+    // optional bool usePersistentCache = 8;
+    public static final int USEPERSISTENTCACHE_FIELD_NUMBER = 8;
+    private boolean usePersistentCache_;
+    /**
+     * <code>optional bool usePersistentCache = 8;</code>
+     */
+    public boolean hasUsePersistentCache() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional bool usePersistentCache = 8;</code>
+     */
+    public boolean getUsePersistentCache() {
+      return usePersistentCache_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       cacheId_ = com.google.protobuf.ByteString.EMPTY;
@@ -6035,6 +6066,7 @@ public final class ServerCachingProtos {
       txState_ = com.google.protobuf.ByteString.EMPTY;
       hasProtoBufIndexMaintainer_ = false;
       clientVersion_ = 0;
+      usePersistentCache_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -6089,6 +6121,9 @@ public final class ServerCachingProtos {
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeInt32(7, clientVersion_);
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBool(8, usePersistentCache_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -6126,6 +6161,10 @@ public final class ServerCachingProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(7, clientVersion_);
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(8, usePersistentCache_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -6184,6 +6223,11 @@ public final class ServerCachingProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasUsePersistentCache() == other.hasUsePersistentCache());
+      if (hasUsePersistentCache()) {
+        result = result && (getUsePersistentCache()
+            == other.getUsePersistentCache());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -6225,6 +6269,10 @@ public final class ServerCachingProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasUsePersistentCache()) {
+        hash = (37 * hash) + USEPERSISTENTCACHE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getUsePersistentCache());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -6358,6 +6406,8 @@ public final class ServerCachingProtos {
         bitField0_ = (bitField0_ & ~0x00000020);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000040);
+        usePersistentCache_ = false;
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -6422,6 +6472,10 @@ public final class ServerCachingProtos {
           to_bitField0_ |= 0x00000040;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.usePersistentCache_ = usePersistentCache_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -6459,6 +6513,9 @@ public final class ServerCachingProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasUsePersistentCache()) {
+          setUsePersistentCache(other.getUsePersistentCache());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -6914,6 +6971,39 @@ public final class ServerCachingProtos {
         return this;
       }
 
+      // optional bool usePersistentCache = 8;
+      private boolean usePersistentCache_ ;
+      /**
+       * <code>optional bool usePersistentCache = 8;</code>
+       */
+      public boolean hasUsePersistentCache() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional bool usePersistentCache = 8;</code>
+       */
+      public boolean getUsePersistentCache() {
+        return usePersistentCache_;
+      }
+      /**
+       * <code>optional bool usePersistentCache = 8;</code>
+       */
+      public Builder setUsePersistentCache(boolean value) {
+        bitField0_ |= 0x00000080;
+        usePersistentCache_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool usePersistentCache = 8;</code>
+       */
+      public Builder clearUsePersistentCache() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        usePersistentCache_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:AddServerCacheRequest)
     }
 
@@ -8723,22 +8813,23 @@ public final class ServerCachingProtos {
       "\timmutable\030\022 \002(\010\022&\n\021indexedColumnInfo\030\023 " +
       "\003(\0132\013.ColumnInfo\022\026\n\016encodingScheme\030\024 \002(\005" +
       "\022\036\n\026immutableStorageScheme\030\025 \002(\005\022\025\n\rview" +
-      "IndexType\030\026 \001(\005\"\334\001\n\025AddServerCacheReques" +
+      "IndexType\030\026 \001(\005\"\370\001\n\025AddServerCacheReques" +
       "t\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014\022)\n\010" +
       "cachePtr\030\003 \002(\0132\027.ImmutableBytesWritable\022" +
       ")\n\014cacheFactory\030\004 \002(\0132\023.ServerCacheFacto" +
       "ry\022\017\n\007txState\030\005 \001(\014\022\"\n\032hasProtoBufIndexM" +
-      "aintainer\030\006 \001(\010\022\025\n\rclientVersion\030\007 \001(\005\"(",
-      "\n\026AddServerCacheResponse\022\016\n\006return\030\001 \002(\010" +
-      "\"=\n\030RemoveServerCacheRequest\022\020\n\010tenantId" +
-      "\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014\"+\n\031RemoveServerC" +
-      "acheResponse\022\016\n\006return\030\001 \002(\0102\245\001\n\024ServerC" +
-      "achingService\022A\n\016addServerCache\022\026.AddSer" +
-      "verCacheRequest\032\027.AddServerCacheResponse" +
-      "\022J\n\021removeServerCache\022\031.RemoveServerCach" +
-      "eRequest\032\032.RemoveServerCacheResponseBG\n(" +
-      "org.apache.phoenix.coprocessor.generated" +
-      "B\023ServerCachingProtosH\001\210\001\001\240\001\001"
+      "aintainer\030\006 \001(\010\022\025\n\rclientVersion\030\007 \001(\005\022\032",
+      "\n\022usePersistentCache\030\010 \001(\010\"(\n\026AddServerC" +
+      "acheResponse\022\016\n\006return\030\001 \002(\010\"=\n\030RemoveSe" +
+      "rverCacheRequest\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007ca" +
+      "cheId\030\002 \002(\014\"+\n\031RemoveServerCacheResponse" +
+      "\022\016\n\006return\030\001 \002(\0102\245\001\n\024ServerCachingServic" +
+      "e\022A\n\016addServerCache\022\026.AddServerCacheRequ" +
+      "est\032\027.AddServerCacheResponse\022J\n\021removeSe" +
+      "rverCache\022\031.RemoveServerCacheRequest\032\032.R" +
+      "emoveServerCacheResponseBG\n(org.apache.p" +
+      "hoenix.coprocessor.generatedB\023ServerCach",
+      "ingProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8774,7 +8865,7 @@ public final class ServerCachingProtos {
           internal_static_AddServerCacheRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_AddServerCacheRequest_descriptor,
-              new java.lang.String[] { "TenantId", "CacheId", "CachePtr", "CacheFactory", "TxState", "HasProtoBufIndexMaintainer", "ClientVersion", });
+              new java.lang.String[] { "TenantId", "CacheId", "CachePtr", "CacheFactory", "TxState", "HasProtoBufIndexMaintainer", "ClientVersion", "UsePersistentCache", });
           internal_static_AddServerCacheResponse_descriptor =
             getDescriptor().getMessageTypes().get(5);
           internal_static_AddServerCacheResponse_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
index bfe089d..b5cd6b1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
@@ -22,21 +22,23 @@ import static org.apache.phoenix.util.LogUtil.addCustomAnnotations;
 import static org.apache.phoenix.util.NumberUtil.add;
 import static org.apache.phoenix.util.NumberUtil.getMin;
 
+import java.io.IOException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
 import java.sql.SQLException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
 import org.apache.phoenix.compile.ColumnProjector;
 import org.apache.phoenix.compile.ExplainPlan;
@@ -46,6 +48,7 @@ import org.apache.phoenix.compile.RowProjector;
 import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.WhereCompiler;
+import org.apache.phoenix.coprocessor.HashJoinCacheNotFoundException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.visitor.AvgRowWidthVisitor;
@@ -57,9 +60,7 @@ import org.apache.phoenix.expression.InListExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
-import org.apache.phoenix.iterate.FilterResultIterator;
-import org.apache.phoenix.iterate.ParallelScanGrouper;
-import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.*;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.job.JobManager.JobCallable;
 import org.apache.phoenix.join.HashCacheClient;
@@ -86,9 +87,11 @@ import org.apache.phoenix.util.SQLCloseables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.apache.phoenix.util.ServerUtil;
 
 public class HashJoinPlan extends DelegateQueryPlan {
     private static final Log LOG = LogFactory.getLog(HashJoinPlan.class);
+    private static final Random RANDOM = new Random();
 
     private final SelectStatement statement;
     private final HashJoinInfo joinInfo;
@@ -105,6 +108,7 @@ public class HashJoinPlan extends DelegateQueryPlan {
     private Long estimatedBytes;
     private Long estimateInfoTs;
     private boolean getEstimatesCalled;
+    private boolean hasSubPlansWithPersistentCache;
     
     public static HashJoinPlan create(SelectStatement statement, 
             QueryPlan plan, HashJoinInfo joinInfo, SubPlan[] subPlans) throws SQLException {
@@ -134,8 +138,12 @@ public class HashJoinPlan extends DelegateQueryPlan {
         this.recompileWhereClause = recompileWhereClause;
         this.tableRefs = Sets.newHashSetWithExpectedSize(subPlans.length + plan.getSourceRefs().size());
         this.tableRefs.addAll(plan.getSourceRefs());
+        this.hasSubPlansWithPersistentCache = false;
         for (SubPlan subPlan : subPlans) {
             tableRefs.addAll(subPlan.getInnerPlan().getSourceRefs());
+            if (subPlan instanceof HashSubPlan && ((HashSubPlan)subPlan).usePersistentCache) {
+                this.hasSubPlansWithPersistentCache = true;
+            }
         }
         QueryServices services = plan.getContext().getConnection().getQueryServices();
         this.maxServerCacheTimeToLive = services.getProps().getInt(
@@ -214,7 +222,7 @@ public class HashJoinPlan extends DelegateQueryPlan {
             SQLCloseables.closeAllQuietly(dependencies.values());
             throw firstException;
         }
-        
+
         Expression postFilter = null;
         boolean hasKeyRangeExpressions = keyRangeExpressions != null && !keyRangeExpressions.isEmpty();
         if (recompileWhereClause || hasKeyRangeExpressions) {
@@ -241,8 +249,35 @@ public class HashJoinPlan extends DelegateQueryPlan {
         if (statement.getInnerSelectStatement() != null && postFilter != null) {
             iterator = new FilterResultIterator(iterator, postFilter);
         }
-        
-        return iterator;
+
+        if (hasSubPlansWithPersistentCache) {
+            return peekForPersistentCache(iterator, scanGrouper, scan);
+        } else {
+            return iterator;
+        }
+    }
+
+    private ResultIterator peekForPersistentCache(ResultIterator iterator, ParallelScanGrouper scanGrouper, Scan scan) throws SQLException {
+        // The persistent subquery is optimistic and assumes caches are present on region
+        // servers. We verify that this is the case by peeking at one result. If there is
+        // a cache missing exception, we retry the query with the persistent cache disabled
+        // for that specific cache ID.
+        PeekingResultIterator peeking = LookAheadResultIterator.wrap(iterator);
+        try {
+            peeking.peek();
+        } catch (Exception e) {
+            try {
+                throw ServerUtil.parseServerException(e);
+            } catch (HashJoinCacheNotFoundException e2) {
+                Long cacheId = e2.getCacheId();
+                if (delegate.getContext().getRetryingPersistentCache(cacheId)) {
+                    throw e2;
+                }
+                delegate.getContext().setRetryingPersistentCache(cacheId);
+                return iterator(scanGrouper, scan);
+            }
+        }
+        return peeking;
     }
 
     private Expression createKeyRangeExpression(Expression lhsExpression,
@@ -467,20 +502,29 @@ public class HashJoinPlan extends DelegateQueryPlan {
         private final QueryPlan plan;
         private final List<Expression> hashExpressions;
         private final boolean singleValueOnly;
+        private final boolean usePersistentCache;
         private final Expression keyRangeLhsExpression;
         private final Expression keyRangeRhsExpression;
+        private final MessageDigest digest;
         
         public HashSubPlan(int index, QueryPlan subPlan, 
                 List<Expression> hashExpressions,
                 boolean singleValueOnly,
+                boolean usePersistentCache,
                 Expression keyRangeLhsExpression, 
                 Expression keyRangeRhsExpression) {
             this.index = index;
             this.plan = subPlan;
             this.hashExpressions = hashExpressions;
             this.singleValueOnly = singleValueOnly;
+            this.usePersistentCache = usePersistentCache;
             this.keyRangeLhsExpression = keyRangeLhsExpression;
             this.keyRangeRhsExpression = keyRangeRhsExpression;
+            try {
+                this.digest = MessageDigest.getInstance("SHA-256");
+            } catch (NoSuchAlgorithmException e) {
+                throw new RuntimeException(e);
+            }
         }
 
         @Override
@@ -494,19 +538,37 @@ public class HashJoinPlan extends DelegateQueryPlan {
             if (hashExpressions != null) {
                 ResultIterator iterator = plan.iterator();
                 try {
-                    cache =
-                            parent.hashClient.addHashCache(ranges, iterator,
-                                plan.getEstimatedSize(), hashExpressions, singleValueOnly,
+                    final byte[] cacheId;
+                    String queryString = plan.getStatement().toString().replaceAll("\\$[0-9]+", "\\$");
+                    if (usePersistentCache) {
+                        cacheId = Arrays.copyOfRange(digest.digest(queryString.getBytes()), 0, 8);
+                        boolean retrying = parent.delegate.getContext().getRetryingPersistentCache(Bytes.toLong(cacheId));
+                        if (!retrying) {
+                            try {
+                                cache = parent.hashClient.createServerCache(cacheId, parent.delegate);
+                            } catch (IOException e) {
+                                throw new RuntimeException(e);
+                            }
+                        }
+                    } else {
+                        cacheId = Bytes.toBytes(RANDOM.nextLong());
+                    }
+                    LOG.debug("Using cache ID " + Hex.encodeHexString(cacheId) + " for " + queryString);
+                    if (cache == null) {
+                        LOG.debug("Making RPC to add cache " + Hex.encodeHexString(cacheId));
+                        cache = parent.hashClient.addHashCache(ranges, cacheId, iterator,
+                                plan.getEstimatedSize(), hashExpressions, singleValueOnly, usePersistentCache,
                                 parent.delegate.getTableRef().getTable(), keyRangeRhsExpression,
                                 keyRangeRhsValues);
-                    long endTime = System.currentTimeMillis();
-                    boolean isSet = parent.firstJobEndTime.compareAndSet(0, endTime);
-                    if (!isSet && (endTime
-                            - parent.firstJobEndTime.get()) > parent.maxServerCacheTimeToLive) {
-                        LOG.warn(addCustomAnnotations(
-                            "Hash plan [" + index
-                                    + "] execution seems too slow. Earlier hash cache(s) might have expired on servers.",
-                            parent.delegate.getContext().getConnection()));
+                        long endTime = System.currentTimeMillis();
+                        boolean isSet = parent.firstJobEndTime.compareAndSet(0, endTime);
+                        if (!isSet && (endTime
+                                - parent.firstJobEndTime.get()) > parent.maxServerCacheTimeToLive) {
+                            LOG.warn(addCustomAnnotations(
+                                "Hash plan [" + index
+                                        + "] execution seems too slow. Earlier hash cache(s) might have expired on servers.",
+                                parent.delegate.getContext().getConnection()));
+                        }
                     }
                 } finally {
                     iterator.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 d890383..2378175 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
@@ -1309,8 +1309,12 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                                     throw e2;
                                 }
                                 Long cacheId = ((HashJoinCacheNotFoundException)e2).getCacheId();
-                                if (!hashCacheClient.addHashCacheToServer(startKey,
-                                        caches.get(new ImmutableBytesPtr(Bytes.toBytes(cacheId))), plan.getTableRef().getTable())) { throw e2; }
+                                ServerCache cache = caches.get(new ImmutableBytesPtr(Bytes.toBytes(cacheId)));
+                                if (cache .getCachePtr() != null) {
+                                    if (!hashCacheClient.addHashCacheToServer(startKey, cache, plan.getTableRef().getTable())) {
+                                        throw e2;
+                                    }
+                                }
                             }
                             concatIterators =
                                     recreateIterators(services, isLocalIndex, allIterators,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
index 06f612a..f1d1663 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
@@ -194,13 +194,11 @@ public class TableResultIterator implements ResultIterator {
                             if (retry <= 0) {
                                 throw e1;
                             }
+                            Long cacheId = ((HashJoinCacheNotFoundException) e1).getCacheId();
                             retry--;
                             try {
-                                Long cacheId = ((HashJoinCacheNotFoundException) e1).getCacheId();
-
                                 ServerCache cache = caches == null ? null :
-                                    caches.get(new ImmutableBytesPtr(Bytes.toBytes(cacheId)));
-
+                                        caches.get(new ImmutableBytesPtr(Bytes.toBytes(cacheId)));
                                 if (!hashCacheClient.addHashCacheToServer(newScan.getStartRow(),
                                         cache, plan.getTableRef().getTable())) {
                                     throw e1;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
index 83ac32d..315c515 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.cache.ServerCacheClient;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
@@ -67,6 +68,20 @@ public class HashCacheClient  {
     }
 
     /**
+     * Creates a ServerCache object for cacheId. This is used for persistent cache, and there may or may not
+     * be corresponding data on each region server.
+     * @param cacheId ID for the cache entry
+     * @param delegate the query plan this will be used for
+     * @return client-side {@link ServerCache} representing the hash cache that may or may not be present on region servers.
+     * @throws SQLException
+     * size
+     */
+    public ServerCache createServerCache(final byte[] cacheId, QueryPlan delegate)
+            throws SQLException, IOException {
+        return serverCache.createServerCache(cacheId, delegate);
+    }
+
+    /**
      * Send the results of scanning through the scanner to all
      * region servers for regions of the table that will use the cache
      * that intersect with the minMaxKeyRange.
@@ -76,13 +91,16 @@ public class HashCacheClient  {
      * @throws MaxServerCacheSizeExceededException if size of hash cache exceeds max allowed
      * size
      */
-    public ServerCache addHashCache(ScanRanges keyRanges, ResultIterator iterator, long estimatedSize, List<Expression> onExpressions, boolean singleValueOnly, PTable cacheUsingTable, Expression keyRangeRhsExpression, List<Expression> keyRangeRhsValues) throws SQLException {
+    public ServerCache addHashCache(
+            ScanRanges keyRanges, byte[] cacheId, ResultIterator iterator, long estimatedSize, List<Expression> onExpressions,
+            boolean singleValueOnly, boolean usePersistentCache, PTable cacheUsingTable, Expression keyRangeRhsExpression,
+            List<Expression> keyRangeRhsValues) throws SQLException {
         /**
          * Serialize and compress hashCacheTable
          */
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         serialize(ptr, iterator, estimatedSize, onExpressions, singleValueOnly, keyRangeRhsExpression, keyRangeRhsValues);
-        ServerCache cache = serverCache.addServerCache(keyRanges, ptr, ByteUtil.EMPTY_BYTE_ARRAY, new HashCacheFactory(), cacheUsingTable, true);
+        ServerCache cache = serverCache.addServerCache(keyRanges, cacheId, ptr, ByteUtil.EMPTY_BYTE_ARRAY, new HashCacheFactory(), cacheUsingTable, usePersistentCache, true);
         return cache;
     }
     
@@ -90,7 +108,7 @@ public class HashCacheClient  {
      * Should only be used to resend the hash table cache to the regionserver.
      *  
      * @param startkeyOfRegion start key of any region hosted on a regionserver which needs hash cache
-     * @param cacheId Id of the cache which needs to be sent
+     * @param cache The cache which needs to be sent
      * @param pTable
      * @return
      * @throws Exception

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
index 4fc3c70..ecf9d57 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
@@ -30,6 +30,7 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import net.jcip.annotations.Immutable;
 
@@ -139,6 +140,16 @@ public class HashCacheFactory implements ServerCacheFactory {
         }
 
         @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            Set<ImmutableBytesPtr> keySet = hashCache.keySet();
+            for (ImmutableBytesPtr key : keySet) {
+                sb.append("key: " + key + " value: " + hashCache.get(key));
+            }
+            return sb.toString();
+        }
+
+        @Override
         public void close() {
             memoryChunk.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 02a44ad..8a83116 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
@@ -83,6 +83,10 @@ public class HintNode {
         */
        USE_SORT_MERGE_JOIN,
        /**
+        * Persist the RHS results of a hash join.
+        */
+       USE_PERSISTENT_CACHE,
+       /**
         * Avoid using star-join optimization. Used for broadcast join (hash join) only.
         */
        NO_STAR_JOIN,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 d681a13..d1b277a 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
@@ -90,6 +90,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String MUTATE_BATCH_SIZE_ATTRIB = "phoenix.mutate.batchSize";
     public static final String MUTATE_BATCH_SIZE_BYTES_ATTRIB = "phoenix.mutate.batchSizeBytes";
     public static final String MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxServerCacheTimeToLiveMs";
+    public static final String MAX_SERVER_CACHE_PERSISTENCE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxServerCachePersistenceTimeToLiveMs";
     
     @Deprecated // Use FORCE_ROW_KEY_ORDER instead.
     public static final String ROW_KEY_ORDER_SALTED_TABLE_ATTRIB  = "phoenix.query.rowKeyOrderSaltedTable";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/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 76e79fa..35dbe3a 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
@@ -159,6 +159,7 @@ public class QueryServicesOptions {
     public final static long DEFAULT_MUTATE_BATCH_SIZE_BYTES = 2097152;
 	// The only downside of it being out-of-sync is that the parallelization of the scan won't be as balanced as it could be.
     public static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 30000; // 30 sec (with no activity)
+    public static final int DEFAULT_MAX_SERVER_CACHE_PERSISTENCE_TIME_TO_LIVE_MS = 30 * 60000; // 30 minutes
     public static final int DEFAULT_SCAN_CACHE_SIZE = 1000;
     public static final int DEFAULT_MAX_INTRA_REGION_PARALLELIZATION = DEFAULT_MAX_QUERY_CONCURRENCY;
     public static final int DEFAULT_DISTINCT_VALUE_COMPRESS_THRESHOLD = 1024 * 1024 * 1; // 1 Mb

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java b/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
index c741f4e..3c8a269 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
@@ -17,9 +17,6 @@
  */
 package org.apache.phoenix.cache;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
 import java.io.Closeable;
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -38,49 +35,53 @@ import org.junit.Test;
 
 import com.google.common.base.Ticker;
 
+import static org.junit.Assert.*;
+
 public class TenantCacheTest {
 
     @Test
     public void testInvalidateClosesMemoryChunk() throws SQLException {
         int maxServerCacheTimeToLive = 10000;
+        int maxServerCachePersistenceTimeToLive = 10;
         long maxBytes = 1000;
         GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
-        TenantCacheImpl newTenantCache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive);
-        ImmutableBytesPtr cacheId = new ImmutableBytesPtr(Bytes.toBytes("a"));
+        TenantCacheImpl newTenantCache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive);
+        ImmutableBytesPtr cacheId = new ImmutableBytesPtr(Bytes.toBytes(1L));
         ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("a"));
-        newTenantCache.addServerCache(cacheId, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, MetaDataProtocol.PHOENIX_VERSION);
+        newTenantCache.addServerCache(cacheId, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, false, MetaDataProtocol.PHOENIX_VERSION);
         assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
         newTenantCache.removeServerCache(cacheId);
         assertEquals(maxBytes, memoryManager.getAvailableMemory());
     }
-    
+
     @Test
     public void testTimeoutClosesMemoryChunk() throws Exception {
         int maxServerCacheTimeToLive = 10;
+        int maxServerCachePersistenceTimeToLive = 10;
         long maxBytes = 1000;
         GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
         ManualTicker ticker = new ManualTicker();
-        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, ticker);
-        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes("a"));
+        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive, ticker);
+        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes(1L));
         ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("a"));
-        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, MetaDataProtocol.PHOENIX_VERSION);
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, false, MetaDataProtocol.PHOENIX_VERSION);
         assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
         ticker.time += (maxServerCacheTimeToLive + 1) * 1000000;
         cache.cleanUp();
         assertEquals(maxBytes, memoryManager.getAvailableMemory());
     }
 
-
     @Test
     public void testFreeMemoryOnAccess() throws Exception {
         int maxServerCacheTimeToLive = 10;
+        int maxServerCachePersistenceTimeToLive = 10;
         long maxBytes = 1000;
         GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
         ManualTicker ticker = new ManualTicker();
-        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, ticker);
-        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes("a"));
+        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive, ticker);
+        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes(1L));
         ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("a"));
-        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, MetaDataProtocol.PHOENIX_VERSION);
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, false, MetaDataProtocol.PHOENIX_VERSION);
         assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
         ticker.time += (maxServerCacheTimeToLive + 1) * 1000000;
         assertNull(cache.getServerCache(cacheId1));
@@ -90,17 +91,92 @@ public class TenantCacheTest {
     @Test
     public void testExpiredCacheOnAddingNew() throws Exception {
         int maxServerCacheTimeToLive = 10;
+        int maxServerCachePersistenceTimeToLive = 10;
         long maxBytes = 10;
         GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
         ManualTicker ticker = new ManualTicker();
-        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, ticker);
-        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes("a"));
+        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive, ticker);
+        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes(1L));
         ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("12345678"));
-        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, MetaDataProtocol.PHOENIX_VERSION);
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, false, MetaDataProtocol.PHOENIX_VERSION);
         assertEquals(2, memoryManager.getAvailableMemory());
         ticker.time += (maxServerCacheTimeToLive + 1) * 1000000;
-        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, MetaDataProtocol.PHOENIX_VERSION);
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, false, MetaDataProtocol.PHOENIX_VERSION);
+        assertEquals(2, memoryManager.getAvailableMemory());
+    }
+
+    @Test
+    public void testExpiresButStaysInPersistentAfterTimeout() throws Exception {
+        int maxServerCacheTimeToLive = 100;
+        int maxServerCachePersistenceTimeToLive = 1000;
+        long maxBytes = 1000;
+        GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
+        ManualTicker ticker = new ManualTicker();
+        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive, ticker);
+        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes(1L));
+        ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("a"));
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, true, MetaDataProtocol.PHOENIX_VERSION);
+        assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
+        assertNotNull(cache.getServerCache(cacheId1));
+
+        // Expire it from live cache but not persistent cache
+        ticker.time += (maxServerCacheTimeToLive + 1) * 1000000;
+        cache.cleanUp();
+        assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
+        assertNotNull(cache.getServerCache(cacheId1));
+
+        // Expire it from persistent cache as well
+        ticker.time += (maxServerCachePersistenceTimeToLive + 1) * 1000000;
+        cache.cleanUp();
+        assertEquals(maxBytes, memoryManager.getAvailableMemory());
+        assertNull(cache.getServerCache(cacheId1));
+    }
+
+    @Test
+    public void testExpiresButStaysInPersistentAfterRemove() throws Exception {
+        int maxServerCacheTimeToLive = 100;
+        int maxServerCachePersistenceTimeToLive = 1000;
+        long maxBytes = 1000;
+        GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
+        ManualTicker ticker = new ManualTicker();
+        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive, ticker);
+        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes(1L));
+        ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("12"));
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, true, MetaDataProtocol.PHOENIX_VERSION);
+        assertEquals(maxBytes-2, memoryManager.getAvailableMemory());
+        assertNotNull(cache.getServerCache(cacheId1));
+
+        // Remove should only remove from live cache
+        cache.removeServerCache(cacheId1);
+        assertEquals(maxBytes-2, memoryManager.getAvailableMemory());
+        assertNotNull(cache.getServerCache(cacheId1));
+    }
+
+    @Test
+    public void testEvictPersistentCacheIfSpaceIsNeeded() throws Exception {
+        int maxServerCacheTimeToLive = 100;
+        int maxServerCachePersistenceTimeToLive = 1000;
+        long maxBytes = 10;
+        GlobalMemoryManager memoryManager = new GlobalMemoryManager(maxBytes);
+        ManualTicker ticker = new ManualTicker();
+        TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, maxServerCachePersistenceTimeToLive, ticker);
+        ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes(1L));
+        ImmutableBytesWritable cachePtr1 = new ImmutableBytesWritable(Bytes.toBytes("1234"));
+        cache.addServerCache(cacheId1, cachePtr1, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, true, MetaDataProtocol.PHOENIX_VERSION);
+        assertEquals(6, memoryManager.getAvailableMemory());
+
+        // Remove it, but it should stay in persistent cache
+        cache.removeServerCache(cacheId1);
+        assertNotNull(cache.getServerCache(cacheId1));
+        assertEquals(6, memoryManager.getAvailableMemory());
+
+        // Let's do an entry that will require eviction
+        ImmutableBytesPtr cacheId2 = new ImmutableBytesPtr(Bytes.toBytes(2L));
+        ImmutableBytesWritable cachePtr2 = new ImmutableBytesWritable(Bytes.toBytes("12345678"));
+        cache.addServerCache(cacheId2, cachePtr2, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true, true, MetaDataProtocol.PHOENIX_VERSION);
         assertEquals(2, memoryManager.getAvailableMemory());
+        assertNull(cache.getServerCache(cacheId1));
+        assertNotNull(cache.getServerCache(cacheId2));
     }
 
     public static class ManualTicker extends Ticker {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-protocol/src/main/ServerCachingService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/ServerCachingService.proto b/phoenix-protocol/src/main/ServerCachingService.proto
index d92f2cd..0d2d1d2 100644
--- a/phoenix-protocol/src/main/ServerCachingService.proto
+++ b/phoenix-protocol/src/main/ServerCachingService.proto
@@ -73,6 +73,7 @@ message AddServerCacheRequest {
   optional bytes txState = 5;
   optional bool hasProtoBufIndexMaintainer = 6;
   optional int32 clientVersion = 7;
+  optional bool usePersistentCache = 8;
 }
 
 message AddServerCacheResponse {


[24/50] [abbrv] phoenix git commit: PHOENIX-4830 fix order by primary key desc

Posted by ja...@apache.org.
PHOENIX-4830 fix order by primary key desc

add unit test


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

Branch: refs/heads/omid2
Commit: 59f8d0fd1a110786251dbf79c7bc743d1569b54c
Parents: 913a855
Author: Xu Cang <xc...@salesforce.com>
Authored: Fri Aug 10 00:45:17 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Sat Aug 11 11:43:14 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/OrderByIT.java   | 46 ++++++++++++++++++++
 .../phoenix/iterate/TableResultIterator.java    | 10 ++++-
 .../java/org/apache/phoenix/util/ScanUtil.java  | 19 ++++++++
 3 files changed, 73 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/59f8d0fd/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
index 578a3af..d7bbc05 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
@@ -656,6 +656,52 @@ public class OrderByIT extends ParallelStatsDisabledIT {
     }
 
     @Test
+    public void testOrderByDescOnPkWithSubQuery() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        String ddl = "create table " + tableName + " (id bigint not null primary key, a bigint)";
+        conn.createStatement().execute(ddl);
+
+        conn.createStatement().execute("upsert into " + tableName + " values (1, 11)");
+        conn.createStatement().execute("upsert into " + tableName + " values (2, 22)");
+        conn.createStatement().execute("upsert into " + tableName + " values (3, 33)");
+        conn.createStatement().execute("upsert into " + tableName + " values (4,44)");
+
+        conn.commit();
+
+        ResultSet rs;
+        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
+        rs = stmt.executeQuery("select id from " + tableName + " where id in (select id from "
+            + tableName + ") order by id desc");
+        assertTrue(rs.next());
+        assertEquals("4", rs.getString(1));
+    }
+
+    @Test
+    public void testOrderByAscOnPkWithSubQuery() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateUniqueName();
+        String ddl = "create table " + tableName + " (id bigint not null primary key, a bigint)";
+        conn.createStatement().execute(ddl);
+
+        conn.createStatement().execute("upsert into " + tableName + " values (1, 11)");
+        conn.createStatement().execute("upsert into " + tableName + " values (2, 22)");
+        conn.createStatement().execute("upsert into " + tableName + " values (3, 33)");
+        conn.createStatement().execute("upsert into " + tableName + " values (4,44)");
+
+        conn.commit();
+
+        ResultSet rs;
+        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
+        rs = stmt.executeQuery("select id from " + tableName + " where id in (select id from "
+            + tableName + ") order by id");
+        assertTrue(rs.next());
+        assertEquals("1", rs.getString(1));
+    }
+
+    @Test
     public void testNullsLastWithDesc() throws Exception {
         Connection conn=null;
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59f8d0fd/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
index 06f612a..9a80d9f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.iterate;
 
+import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.REVERSE_SCAN;
 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.iterate.TableResultIterator.RenewLeaseStatus.CLOSED;
@@ -53,6 +54,7 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
@@ -246,8 +248,12 @@ public class TableResultIterator implements ResultIterator {
             ResultIterator delegate = this.scanIterator;
             if (delegate == UNINITIALIZED_SCANNER) {
                 try {
-                    this.scanIterator =
-                            new ScanningResultIterator(htable.getScanner(scan), scan, scanMetricsHolder);
+                    if (scan.getAttribute(REVERSE_SCAN) != null
+                        && (boolean)(PBoolean.INSTANCE.toObject(scan.getAttribute(REVERSE_SCAN)))) {
+                        ScanUtil.prepareStopRowForReverseScan(scan);
+                    }
+                    this.scanIterator = new ScanningResultIterator(htable.getScanner(scan), scan,
+                        scanMetricsHolder);
                 } catch (IOException e) {
                     Closeables.closeQuietly(htable);
                     throw ServerUtil.parseServerException(e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59f8d0fd/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 62ecebd..2dd46a6 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
@@ -656,6 +656,25 @@ public class ScanUtil {
     }
 
     /**
+     * HBase scan stopRow is exclusive.So we have to append trailing 0 to achieve inclusiveness.
+     * for reverse scan, need to append trailing F to stopRow.
+     * @param scan
+     */
+    public static void prepareStopRowForReverseScan(Scan scan) {
+        byte[] stopRow = scan.getStopRow();
+        if (stopRow == null) {
+            return;
+        }
+        byte[] newStopRow = new byte[stopRow.length + 1];
+        int i = 0;
+        for (byte nsr : stopRow) {
+            newStopRow[i++] = nsr;
+        }
+        newStopRow[i] = QueryConstants.DESC_SEPARATOR_BYTE;
+        scan.setStopRow(newStopRow);
+    }
+
+    /**
      * Start key and stop key of the original scan from client are regions start and end keys so
      * prefix scan start/stop key to the start row/stop row suffix and set them as scan boundaries.
      * @param scan


[02/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 b127408..9d5583b 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
@@ -82,12 +82,12 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 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.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.LocalIndexSplitter;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -96,6 +96,9 @@ import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
 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.TableInfo;
+import org.apache.phoenix.coprocessor.TableViewFinderResult;
+import org.apache.phoenix.coprocessor.ViewFinder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
@@ -175,11 +178,6 @@ public class UpgradeUtil {
     private static final String DELETE_LINK = "DELETE FROM " + SYSTEM_CATALOG_SCHEMA + "." + SYSTEM_CATALOG_TABLE
             + " WHERE (" + TABLE_SCHEM + "=? OR (" + TABLE_SCHEM + " IS NULL AND ? IS NULL)) AND " + TABLE_NAME + "=? AND " + COLUMN_FAMILY + "=? AND " + LINK_TYPE + " = " + LinkType.PHYSICAL_TABLE.getSerializedValue();
     
-    private static final String GET_VIEWS_QUERY = "SELECT " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME
-            + " FROM " + SYSTEM_CATALOG_SCHEMA + "." + SYSTEM_CATALOG_TABLE + " WHERE " + COLUMN_FAMILY + " = ? AND "
-            + LINK_TYPE + " = " + LinkType.PHYSICAL_TABLE.getSerializedValue() + " AND ( " + TABLE_TYPE + "=" + "'"
-            + PTableType.VIEW.getSerializedValue() + "' OR " + TABLE_TYPE + " IS NULL) ORDER BY "+TENANT_ID;
-    
     private UpgradeUtil() {
     }
 
@@ -225,8 +223,8 @@ public class UpgradeUtil {
         scan.setRaw(true);
         scan.setMaxVersions();
         ResultScanner scanner = null;
-        HTableInterface source = null;
-        HTableInterface target = null;
+        Table source = null;
+        Table target = null;
         try {
             source = conn.getQueryServices().getTable(sourceName);
             target = conn.getQueryServices().getTable(targetName);
@@ -646,7 +644,7 @@ public class UpgradeUtil {
         logger.info("Upgrading SYSTEM.SEQUENCE table");
 
         byte[] seqTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE);
-        HTableInterface sysTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+        Table sysTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
         try {
             logger.info("Setting SALT_BUCKETS property of SYSTEM.SEQUENCE to " + SaltingUtil.MAX_BUCKET_NUM);
             KeyValue saltKV = KeyValueUtil.newKeyValue(seqTableKey, 
@@ -699,7 +697,7 @@ public class UpgradeUtil {
                 Scan scan = new Scan();
                 scan.setRaw(true);
                 scan.setMaxVersions();
-                HTableInterface seqTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
+                Table seqTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                 try {
                     boolean committed = false;
                     logger.info("Adding salt byte to all SYSTEM.SEQUENCE rows");
@@ -1149,6 +1147,78 @@ public class UpgradeUtil {
         }
     }
     
+    /**
+     * Move child links form SYSTEM.CATALOG to SYSTEM.CHILD_LINK
+     * @param oldMetaConnection caller should take care of closing the passed connection appropriately
+     * @throws SQLException
+     */
+    public static void moveChildLinks(PhoenixConnection oldMetaConnection) throws SQLException {
+        PhoenixConnection metaConnection = null;
+        try {
+            // Need to use own connection with max time stamp to be able to read all data from SYSTEM.CATALOG 
+            metaConnection = new PhoenixConnection(oldMetaConnection, HConstants.LATEST_TIMESTAMP);
+            logger.info("Upgrading metadata to add parent to child links for views");
+            metaConnection.commit();
+            String createChildLink = "UPSERT INTO SYSTEM.CHILD_LINK(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, LINK_TYPE) " +
+                                        "SELECT TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, LINK_TYPE " + 
+                                        "FROM SYSTEM.CATALOG " + 
+                                        "WHERE LINK_TYPE = 4";
+            metaConnection.createStatement().execute(createChildLink);
+            metaConnection.commit();
+            String deleteChildLink = "DELETE FROM SYSTEM.CATALOG WHERE LINK_TYPE = 4 ";
+            metaConnection.createStatement().execute(deleteChildLink);
+            metaConnection.commit();
+            metaConnection.getQueryServices().clearCache();
+        } finally {
+            if (metaConnection != null) {
+                metaConnection.close();
+            }
+        }
+    }
+    
+    public static void addViewIndexToParentLinks(PhoenixConnection oldMetaConnection) throws SQLException {
+    	// Need to use own connection with max time stamp to be able to read all data from SYSTEM.CATALOG 
+        try (PhoenixConnection queryConn = new PhoenixConnection(oldMetaConnection, HConstants.LATEST_TIMESTAMP);
+        		PhoenixConnection upsertConn = new PhoenixConnection(oldMetaConnection, HConstants.LATEST_TIMESTAMP)) {
+            logger.info("Upgrading metadata to add parent links for indexes on views");
+			String indexQuery = "SELECT TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_FAMILY FROM SYSTEM.CATALOG WHERE LINK_TYPE = "
+					+ LinkType.INDEX_TABLE.getSerializedValue();
+			String createViewIndexLink = "UPSERT INTO SYSTEM.CATALOG (TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_FAMILY, LINK_TYPE) VALUES (?,?,?,?,?) ";
+            ResultSet rs = queryConn.createStatement().executeQuery(indexQuery);
+            String prevTenantId = null;
+            PhoenixConnection metaConn = queryConn;
+            Properties props = new Properties(queryConn.getClientInfo());
+			props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(HConstants.LATEST_TIMESTAMP));
+            while (rs.next()) {
+            	String tenantId = rs.getString("TENANT_ID");
+				if (prevTenantId != tenantId) {
+					prevTenantId = tenantId;
+					props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+            		metaConn = new PhoenixConnection(oldMetaConnection, props); 
+            	}
+            	String schemaName = rs.getString("TABLE_SCHEM");
+            	String parentTableName = rs.getString("TABLE_NAME");
+            	String fullParentTableName = SchemaUtil.getTableName(schemaName, parentTableName);
+            	String indexName = rs.getString("COLUMN_FAMILY");
+            	PTable table = PhoenixRuntime.getTable(metaConn, fullParentTableName);
+            	if (table==null) {
+            		throw new TableNotFoundException(fullParentTableName);
+            	}
+            	if (table.getType().equals(PTableType.VIEW)) {
+            		PreparedStatement prepareStatement = upsertConn.prepareStatement(createViewIndexLink);
+            		prepareStatement.setString(1, tenantId);
+            		prepareStatement.setString(2, schemaName);
+            		prepareStatement.setString(3, indexName);
+            		prepareStatement.setString(4, parentTableName);
+            		prepareStatement.setByte(5, LinkType.VIEW_INDEX_PARENT_TABLE.getSerializedValue());
+            		prepareStatement.execute();
+            		upsertConn.commit();
+            	}
+            }
+            queryConn.getQueryServices().clearCache();
+        }
+    }
+    
     private static void upsertBaseColumnCountInHeaderRow(PhoenixConnection metaConnection,
             String tenantId, String schemaName, String viewOrTableName, int baseColumnCount)
             throws SQLException {
@@ -1667,7 +1737,7 @@ public class UpgradeUtil {
         tableMetadata.add(put);
     }
 
-    public static boolean truncateStats(HTableInterface metaTable, HTableInterface statsTable)
+    public static boolean truncateStats(Table metaTable, Table statsTable)
             throws IOException, InterruptedException {
         byte[] statsTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
                 PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE);
@@ -1728,7 +1798,7 @@ public class UpgradeUtil {
         return false;
     }
 
-    private static void mapTableToNamespace(HBaseAdmin admin, HTableInterface metatable, String srcTableName,
+    private static void mapTableToNamespace(HBaseAdmin admin, Table metatable, String srcTableName,
             String destTableName, ReadOnlyProps props, Long ts, String phoenixTableName, PTableType pTableType,PName tenantId)
                     throws SnapshotCreationException, IllegalArgumentException, IOException, InterruptedException,
                     SQLException {
@@ -1791,7 +1861,7 @@ public class UpgradeUtil {
      * Method to map existing phoenix table to a namespace. Should not be use if tables has views and indexes ,instead
      * use map table utility in psql.py
      */
-    public static void mapTableToNamespace(HBaseAdmin admin, HTableInterface metatable, String tableName,
+    public static void mapTableToNamespace(HBaseAdmin admin, Table metatable, String tableName,
             ReadOnlyProps props, Long ts, PTableType pTableType, PName tenantId) throws SnapshotCreationException,
                     IllegalArgumentException, IOException, InterruptedException, SQLException {
         String destTablename = SchemaUtil
@@ -1808,14 +1878,15 @@ public class UpgradeUtil {
                 readOnlyProps)) { throw new IllegalArgumentException(
                         QueryServices.IS_NAMESPACE_MAPPING_ENABLED + " is not enabled!!"); }
         try (HBaseAdmin admin = conn.getQueryServices().getAdmin();
-                HTableInterface metatable = conn.getQueryServices()
+                Table metatable = conn.getQueryServices()
                         .getTable(SchemaUtil
                                 .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, readOnlyProps)
                                 .getName());) {
-            String tableName = SchemaUtil.normalizeIdentifier(srcTable);
-            String schemaName = SchemaUtil.getSchemaNameFromFullName(tableName);
+            String fullTableName = SchemaUtil.normalizeIdentifier(srcTable);
+            String schemaName = SchemaUtil.getSchemaNameFromFullName(fullTableName);
+            String tableName = SchemaUtil.getTableNameFromFullName(fullTableName);
             // Confirm table is not already upgraded
-            PTable table = PhoenixRuntime.getTable(conn, tableName);
+            PTable table = PhoenixRuntime.getTable(conn, fullTableName);
             
             // Upgrade is not required if schemaName is not present.
             if (schemaName.equals("") && !PTableType.VIEW
@@ -1829,21 +1900,38 @@ public class UpgradeUtil {
             String oldPhysicalName = table.getPhysicalName().getString();
             String newPhysicalTablename = SchemaUtil.normalizeIdentifier(
                     SchemaUtil.getPhysicalTableName(oldPhysicalName, readOnlyProps).getNameAsString());
-            logger.info(String.format("Upgrading %s %s..", table.getType(), tableName));
+            logger.info(String.format("Upgrading %s %s..", table.getType(), fullTableName));
             logger.info(String.format("oldPhysicalName %s newPhysicalTablename %s..", oldPhysicalName, newPhysicalTablename));
             logger.info(String.format("teanantId %s..", conn.getTenantId()));
+
+            TableViewFinderResult childViewsResult = new TableViewFinderResult();
+            try (Table childLinkTable =
+                    conn.getQueryServices()
+                            .getTable(SchemaUtil.getPhysicalName(
+                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, readOnlyProps)
+                                    .getName())) {
+                byte[] tenantId = conn.getTenantId() != null ? conn.getTenantId().getBytes() : null;
+                ViewFinder.findAllRelatives(childLinkTable, tenantId, schemaName.getBytes(),
+                    tableName.getBytes(), LinkType.CHILD_TABLE, childViewsResult);
+            }
+
             // Upgrade the data or main table
-            mapTableToNamespace(admin, metatable, tableName, newPhysicalTablename, readOnlyProps,
-                    PhoenixRuntime.getCurrentScn(readOnlyProps), tableName, table.getType(),conn.getTenantId());
+            mapTableToNamespace(admin, metatable, fullTableName, newPhysicalTablename, readOnlyProps,
+                    PhoenixRuntime.getCurrentScn(readOnlyProps), fullTableName, table.getType(),conn.getTenantId());
             // clear the cache and get new table
+            conn.removeTable(conn.getTenantId(), fullTableName,
+                table.getParentName() != null ? table.getParentName().getString() : null,
+                table.getTimeStamp());
+            byte[] tenantIdBytes = conn.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : conn.getTenantId().getBytes();
             conn.getQueryServices().clearTableFromCache(
-                    conn.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : conn.getTenantId().getBytes(),
+                    tenantIdBytes,
                     table.getSchemaName().getBytes(), table.getTableName().getBytes(),
                     PhoenixRuntime.getCurrentScn(readOnlyProps));
-            MetaDataMutationResult result = new MetaDataClient(conn).updateCache(conn.getTenantId(),schemaName,
-                    SchemaUtil.getTableNameFromFullName(tableName),true);
+            MetaDataMutationResult result =
+                    new MetaDataClient(conn).updateCache(conn.getTenantId(), schemaName, tableName,
+                        true);
             if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) { throw new TableNotFoundException(
-              schemaName, tableName); }
+              schemaName, fullTableName); }
             table = result.getTable();
             
             // check whether table is properly upgraded before upgrading indexes
@@ -1893,13 +1981,12 @@ public class UpgradeUtil {
                         conn.commit();
                     }
                     conn.getQueryServices().clearTableFromCache(
-                            conn.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : conn.getTenantId().getBytes(),
+                            tenantIdBytes,
                             index.getSchemaName().getBytes(), index.getTableName().getBytes(),
                             PhoenixRuntime.getCurrentScn(readOnlyProps));
                 }
                 updateIndexesSequenceIfPresent(conn, table);
                 conn.commit();
-
             } else {
                 throw new RuntimeException("Error: problem occured during upgrade. Table is not upgraded successfully");
             }
@@ -1907,12 +1994,32 @@ public class UpgradeUtil {
                 logger.info(String.format("Updating link information for view '%s' ..", table.getTableName()));
                 updateLink(conn, oldPhysicalName, newPhysicalTablename,table.getSchemaName(),table.getTableName());
                 conn.commit();
-
+                
+                // if the view is a first level child, then we need to create the PARENT_TABLE link
+                // that was overwritten by the PHYSICAL_TABLE link 
+                if (table.getParentName().equals(table.getPhysicalName())) {
+                    logger.info(String.format("Creating PARENT link for view '%s' ..", table.getTableName()));
+                    // Add row linking view to its parent 
+                    PreparedStatement linkStatement = conn.prepareStatement(MetaDataClient.CREATE_VIEW_LINK);
+                    linkStatement.setString(1, Bytes.toStringBinary(tenantIdBytes));
+                    linkStatement.setString(2, table.getSchemaName().getString());
+                    linkStatement.setString(3, table.getTableName().getString());
+                    linkStatement.setString(4, table.getParentName().getString());
+                    linkStatement.setByte(5, LinkType.PARENT_TABLE.getSerializedValue());
+                    linkStatement.setString(6, null);
+                    linkStatement.execute();
+                    conn.commit();
+                }
+                
                 conn.getQueryServices().clearTableFromCache(
-                    conn.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : conn.getTenantId().getBytes(),
+                    tenantIdBytes,
                     table.getSchemaName().getBytes(), table.getTableName().getBytes(),
                     PhoenixRuntime.getCurrentScn(readOnlyProps));
             }
+            // Upgrade all child views
+            if (table.getType() == PTableType.TABLE) {
+                mapChildViewsToNamespace(conn.getURL(), conn.getClientInfo(), childViewsResult.getLinks());
+            }
         }
     }
 
@@ -1940,7 +2047,7 @@ public class UpgradeUtil {
     private static void updateLink(PhoenixConnection conn, String srcTableName,
             String destTableName, PName schemaName, PName tableName) throws SQLException {
         String updateLinkSql = String.format(UPDATE_LINK, destTableName);
-        boolean hasTenantId = conn.getTenantId() != null;
+        boolean hasTenantId = conn.getTenantId() != null && conn.getTenantId().getBytes().length!=0;
         if (hasTenantId) {
             updateLinkSql += " AND TENANT_ID  = ? ";
         }
@@ -1968,36 +2075,29 @@ public class UpgradeUtil {
         deleteLinkStatment.execute();
     }
     
-    public static void mapChildViewsToNamespace(PhoenixConnection conn, String table, Properties props)
+    private static void mapChildViewsToNamespace(String connUrl, Properties props, List<TableInfo> viewInfoList)
             throws SQLException, SnapshotCreationException, IllegalArgumentException, IOException,
             InterruptedException {
-        PreparedStatement preparedStatment = conn.prepareStatement(GET_VIEWS_QUERY);
-        preparedStatment.setString(1, SchemaUtil.normalizeIdentifier(table));
-        ResultSet rs = preparedStatment.executeQuery();
         String tenantId = null;
         String prevTenantId = null;
-        PhoenixConnection passedConn = conn;
-        while (rs.next()) {
-            tenantId = rs.getString(1);
+        PhoenixConnection conn = null;
+        for (TableInfo viewInfo : viewInfoList) {
+            tenantId = viewInfo.getTenantId()!=null ? Bytes.toString(viewInfo.getTenantId()) : null;
+            String viewName = SchemaUtil.getTableName(viewInfo.getSchemaName(), viewInfo.getTableName());
             if (prevTenantId != tenantId) {
                 if (tenantId != null) {
                     props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
                 } else {
                     props.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                 }
-                if (passedConn != conn) {
+                if (conn!=null)
                     conn.close();
-                }
-                conn = DriverManager.getConnection(conn.getURL(), props).unwrap(PhoenixConnection.class);
+                conn = DriverManager.getConnection(connUrl, props).unwrap(PhoenixConnection.class);
             }
-            String viewName=SchemaUtil.getTableName(rs.getString(2), rs.getString(3));
             logger.info(String.format("Upgrading view %s for tenantId %s..", viewName,tenantId));
             UpgradeUtil.upgradeTable(conn, viewName);
             prevTenantId = tenantId;
         }
-        if (passedConn != conn) {
-            conn.close();
-        }
     }
 
     public static final String getSysCatalogSnapshotName(long currentSystemTableTimestamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 0ff17d3..154dd7a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -1253,7 +1253,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         }
     }
     
-    
+
     @Test
     public void testDuplicatePKColumn() throws Exception {
         String ddl = "CREATE TABLE t (k1 VARCHAR, k1 VARCHAR CONSTRAINT pk PRIMARY KEY(k1))";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
new file mode 100644
index 0000000..cb41191
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
@@ -0,0 +1,299 @@
+package org.apache.phoenix.coprocessor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * 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.
+ */
+public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
+    private final TableName catalogTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+    private final TableName linkTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES);
+
+    /*
+      The tree structure is as follows: Where ParentTable is the Base Table
+      and all children are views and child views respectively.
+
+                ParentTable
+                  /     \
+            leftChild   rightChild
+              /
+       leftGrandChild
+     */
+
+    @Test
+    public void testGettingChildrenAndParentViews() throws Exception {
+        String baseTable = generateUniqueName();
+        String leftChild = generateUniqueName();
+        String rightChild = generateUniqueName();
+        String leftGrandChild = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat =
+            "CREATE TABLE IF NOT EXISTS " + baseTable + "  (" + " PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                + " CONSTRAINT NAME_PK PRIMARY KEY (PK2)" + " )";
+        conn.createStatement().execute(ddlFormat);
+
+        conn.createStatement().execute("CREATE VIEW " + rightChild + " AS SELECT * FROM " + baseTable);
+        conn.createStatement().execute("CREATE VIEW " + leftChild + " (carrier VARCHAR) AS SELECT * FROM " + baseTable);
+        conn.createStatement().execute("CREATE VIEW " + leftGrandChild + " (dropped_calls BIGINT) AS SELECT * FROM " + leftChild);
+
+        PTable table = PhoenixRuntime.getTable(conn, baseTable.toUpperCase());
+        PTable rightChildTable = PhoenixRuntime.getTable(conn, rightChild.toUpperCase());
+        System.err.println(rightChildTable);
+
+        TableViewFinderResult childViews = new TableViewFinderResult();
+        ViewFinder.findAllRelatives(getTable(linkTable), HConstants.EMPTY_BYTE_ARRAY, table.getSchemaName().getBytes(),
+            table.getTableName().getBytes(), PTable.LinkType.CHILD_TABLE, childViews);
+        assertEquals(3, childViews.getLinks().size());
+
+        PTable childMostView = PhoenixRuntime.getTable(conn , leftGrandChild.toUpperCase());
+        TableViewFinderResult parentViews = new TableViewFinderResult();
+        ViewFinder
+            .findAllRelatives(getTable(catalogTable), HConstants.EMPTY_BYTE_ARRAY, childMostView.getSchemaName().getBytes(),
+                childMostView.getTableName().getBytes(), PTable.LinkType.PARENT_TABLE, parentViews);
+        // returns back everything but the parent table - should only return back the left_child and not the right child
+        assertEquals(1, parentViews.getLinks().size());
+        // now lets check and make sure the columns are correct
+        assertColumnNamesEqual(PhoenixRuntime.getTable(conn, childMostView.getName().getString()), "PK2", "V1", "V2", "CARRIER", "DROPPED_CALLS");
+
+    }
+
+    @Test
+    public void testGettingOneChild() throws Exception {
+        String baseTable = generateUniqueName();
+        String leftChild = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat =
+            "CREATE TABLE IF NOT EXISTS " + baseTable + "  (" + " PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                + " CONSTRAINT NAME_PK PRIMARY KEY (PK2)" + " )";
+        conn.createStatement().execute(ddlFormat);
+        conn.createStatement().execute("CREATE VIEW " + leftChild + " (carrier VARCHAR) AS SELECT * FROM " + baseTable);
+
+
+        // now lets check and make sure the columns are correct
+        assertColumnNamesEqual(PhoenixRuntime.getTable(conn, leftChild.toUpperCase()), "PK2", "V1", "V2", "CARRIER");
+    }
+
+    @Test
+    public void testDroppingADerivedColumn() throws Exception {
+        String baseTable = generateUniqueName();
+        String childView = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat = "CREATE TABLE " + baseTable + " (A VARCHAR PRIMARY KEY, B VARCHAR, C VARCHAR)";
+        conn.createStatement().execute(ddlFormat);
+        conn.createStatement().execute("CREATE VIEW " + childView + " (D VARCHAR) AS SELECT * FROM " + baseTable);
+        assertColumnNamesEqual(PhoenixRuntime.getTable(conn, childView.toUpperCase()), "A", "B", "C", "D");
+        conn.createStatement().execute("ALTER VIEW " + childView + " DROP COLUMN C");
+
+        // now lets check and make sure the columns are correct
+        assertColumnNamesEqual(PhoenixRuntime.getTableNoCache(conn, childView.toUpperCase()), "A", "B", "D");
+
+    }
+
+    @Test
+    public void testDroppingAColumn() throws Exception {
+        String baseTable = generateUniqueName();
+        String childView = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat = "CREATE TABLE " + baseTable + " (A VARCHAR PRIMARY KEY, B VARCHAR, C VARCHAR)";
+        conn.createStatement().execute(ddlFormat);
+        conn.createStatement().execute("CREATE VIEW " + childView + " (D VARCHAR) AS SELECT * FROM " + baseTable);
+        assertColumnNamesEqual(PhoenixRuntime.getTable(conn, childView.toUpperCase()), "A", "B", "C", "D");
+        conn.createStatement().execute("ALTER TABLE " + baseTable + " DROP COLUMN C");
+
+        // now lets check and make sure the columns are correct
+        assertColumnNamesEqual(PhoenixRuntime.getTableNoCache(conn, childView.toUpperCase()), "A", "B", "D");
+    }
+
+    @Test
+    public void testAlteringBaseColumns() throws Exception {
+        String baseTable = generateUniqueName();
+        String leftChild = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat =
+            "CREATE TABLE IF NOT EXISTS " + baseTable + "  (" + " PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                + " CONSTRAINT NAME_PK PRIMARY KEY (PK2)" + " )";
+        conn.createStatement().execute(ddlFormat);
+        conn.createStatement().execute("CREATE VIEW " + leftChild + " (carrier VARCHAR) AS SELECT * FROM " + baseTable);
+
+        // now lets check and make sure the columns are correct
+        PTable childPTable = PhoenixRuntime.getTable(conn, leftChild.toUpperCase());
+        assertColumnNamesEqual(childPTable, "PK2", "V1", "V2", "CARRIER");
+
+        // now lets alter the base table by adding a column
+        conn.createStatement().execute("ALTER TABLE " + baseTable + " ADD V3 integer");
+
+        // make sure that column was added to the base table
+        PTable table = PhoenixRuntime.getTableNoCache(conn, baseTable.toUpperCase());
+        assertColumnNamesEqual(table, "PK2", "V1", "V2", "V3");
+
+
+        childPTable = PhoenixRuntime.getTableNoCache(conn, leftChild.toUpperCase());
+        assertColumnNamesEqual(childPTable, "PK2", "V1", "V2", "V3", "CARRIER");
+    }
+
+    @Test
+    public void testAddingAColumnWithADifferentDefinition() throws Exception {
+        String baseTable = generateUniqueName();
+        String view = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat =
+            "CREATE TABLE IF NOT EXISTS " + baseTable + "  (" + " PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                + " CONSTRAINT NAME_PK PRIMARY KEY (PK2)" + " )";
+        conn.createStatement().execute(ddlFormat);
+        conn.createStatement().execute("CREATE VIEW " + view + " (carrier BIGINT) AS SELECT * FROM " + baseTable);
+        Map<String, String> expected = new ImmutableMap.Builder<String, String>()
+            .put("PK2", "VARCHAR")
+            .put("V1", "VARCHAR")
+            .put("V2", "VARCHAR")
+            .put("CARRIER", "BIGINT")
+            .build();
+
+        assertColumnNamesAndDefinitionsEqual(PhoenixRuntime.getTable(conn , view.toUpperCase()), expected);
+        try {
+            conn.createStatement().execute("ALTER TABLE " + baseTable + " ADD carrier VARCHAR");
+        }
+        catch(SQLException e) {
+            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+        }
+
+        Map<String, String> expectedBaseTableColumns = new ImmutableMap.Builder<String, String>()
+            .put("PK2", "VARCHAR")
+            .put("V1", "VARCHAR")
+            .put("V2", "VARCHAR")
+            .build();
+
+        assertColumnNamesAndDefinitionsEqual(PhoenixRuntime.getTable(conn , baseTable.toUpperCase()), expectedBaseTableColumns);
+
+        // the view column "CARRIER" should still be unchanged
+        Map<String, String> expectedViewColumnDefinition = new ImmutableMap.Builder<String, String>()
+            .put("PK2", "VARCHAR")
+            .put("V1", "VARCHAR")
+            .put("V2", "VARCHAR")
+            .put("CARRIER", "BIGINT")
+            .build();
+
+        assertColumnNamesAndDefinitionsEqual(PhoenixRuntime.getTable(conn , view.toUpperCase()), expectedViewColumnDefinition);
+    }
+
+    public void testDropCascade() throws Exception {
+        String baseTable = generateUniqueName();
+        String child = generateUniqueName();
+        String grandChild = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddlFormat =
+            "CREATE TABLE IF NOT EXISTS " + baseTable + "  (" + " PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR "
+                + " CONSTRAINT NAME_PK PRIMARY KEY (PK2)" + " )";
+        conn.createStatement().execute(ddlFormat);
+        conn.createStatement().execute("CREATE VIEW " + child + " (A VARCHAR) AS SELECT * FROM " + baseTable);
+        conn.createStatement().execute("CREATE VIEW " + grandChild + " (B VARCHAR) AS SELECT * FROM " + child);
+
+        PTable childMostView = PhoenixRuntime.getTable(conn , child.toUpperCase());
+        // now lets check and make sure the columns are correct
+        PTable grandChildPTable = PhoenixRuntime.getTable(conn, childMostView.getName().getString());
+        assertColumnNamesEqual(grandChildPTable, "PK2", "V1", "V2", "A");
+
+        // now lets drop the parent table
+        conn.createStatement().execute("DROP TABLE " + baseTable + " CASCADE");
+
+        // the tables should no longer exist
+        try {
+            PhoenixRuntime.getTableNoCache(conn, baseTable);
+            fail();
+        }
+        catch(TableNotFoundException e){}
+        try {
+            PhoenixRuntime.getTableNoCache(conn, child);
+            fail();
+        }
+        catch(TableNotFoundException e){}
+        try {
+            PhoenixRuntime.getTableNoCache(conn, grandChild);
+            fail();
+        }
+        catch(TableNotFoundException e){}
+    }
+
+    @Test
+    public void testWhereClause() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String baseTableName = generateUniqueName();
+        String childViewName = generateUniqueName();
+        String grandChildViewName = generateUniqueName();
+        String baseTableDdl = "CREATE TABLE " + baseTableName + " (" +
+            "A0 CHAR(1) NOT NULL PRIMARY KEY," +
+            "A1 CHAR(1), A2 CHAR (1))";
+        conn.createStatement().execute(baseTableDdl);
+        conn.createStatement().execute(
+            "CREATE VIEW " + childViewName + " AS SELECT * FROM " + baseTableName + " WHERE A1 = 'X'");
+        conn.createStatement().execute(
+            "CREATE VIEW " + grandChildViewName + " AS SELECT * FROM " + childViewName + " WHERE A2 = 'Y'");
+
+        PTable childViewTable = PhoenixRuntime.getTableNoCache(conn, childViewName);
+        PTable grandChildViewTable = PhoenixRuntime.getTableNoCache(conn, grandChildViewName);
+
+        assertNotNull(childViewTable.getColumnForColumnName("A1").getViewConstant());
+        assertNotNull(grandChildViewTable.getColumnForColumnName("A1").getViewConstant());
+        assertNotNull(grandChildViewTable.getColumnForColumnName("A2").getViewConstant());
+    }
+
+    private void assertColumnNamesEqual(PTable table, String... cols) {
+        List<String> actual = Lists.newArrayList();
+        for (PColumn column : table.getColumns()) {
+            actual.add(column.getName().getString().trim());
+        }
+        List<String> expected = Arrays.asList(cols);
+        assertEquals(Joiner.on(", ").join(expected), Joiner.on(", ").join(actual));
+    }
+
+    private void assertColumnNamesAndDefinitionsEqual(PTable table, Map<String, String> expected) {
+        Map<String, String> actual = Maps.newHashMap();
+        for (PColumn column : table.getColumns()) {
+            actual.put(column.getName().getString().trim(), column.getDataType().getSqlTypeName());
+        }
+        assertEquals(expected, actual);
+    }
+
+    private HTable getTable(TableName catalogTable) throws IOException {
+        return new HTable(utility.getConfiguration(), catalogTable);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 d88a915..2e881b8 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
@@ -30,6 +30,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -254,7 +255,7 @@ public class CorrelatePlanTest {
             PName colName = PNameFactory.newName(name);
             columns.add(new PColumnImpl(PNameFactory.newName(name), PNameFactory.newName(VALUE_COLUMN_FAMILY),
                     expr.getDataType(), expr.getMaxLength(), expr.getScale(), expr.isNullable(),
-                    i, expr.getSortOrder(), null, null, false, name, false, false, colName.getBytes()));
+                    i, expr.getSortOrder(), null, null, false, name, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP));
         }
         try {
             PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 017e6c8..6bf298e 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
@@ -30,6 +30,7 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -176,7 +177,8 @@ public class LiteralResultIteratorPlanTest {
             PName colName = PNameFactory.newName(name);
             columns.add(new PColumnImpl(PNameFactory.newName(name),
                     PNameFactory.newName(VALUE_COLUMN_FAMILY), expr.getDataType(), expr.getMaxLength(),
-                    expr.getScale(), expr.isNullable(), i, expr.getSortOrder(), null, null, false, name, false, false, colName.getBytes()));
+                    expr.getScale(), expr.isNullable(), i, expr.getSortOrder(), null, null, false, name, false, false, colName.getBytes(),
+                HConstants.LATEST_TIMESTAMP));
         }
         try {
             PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
index 195c2f0..4808213 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
@@ -31,6 +31,7 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.compile.ColumnResolver;
@@ -120,9 +121,11 @@ public class UnnestArrayPlanTest {
         RowKeyValueAccessor accessor = new RowKeyValueAccessor(Arrays.asList(dummy), 0);
         UnnestArrayPlan plan = new UnnestArrayPlan(subPlan, new RowKeyColumnExpression(dummy, accessor), withOrdinality);
         PName colName = PNameFactory.newName("ELEM");
-        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, colName.getBytes());
+        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, colName.getBytes(),
+            HConstants.LATEST_TIMESTAMP);
         colName = PNameFactory.newName("IDX");
-        PColumn indexColumn = withOrdinality ? new PColumnImpl(colName, PNameFactory.newName(VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, colName.getBytes()) : null;
+        PColumn indexColumn = withOrdinality ? new PColumnImpl(colName, PNameFactory.newName(VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, colName.getBytes(),
+            HConstants.LATEST_TIMESTAMP) : null;
         List<PColumn> columns = withOrdinality ? Arrays.asList(elemColumn, indexColumn) : Arrays.asList(elemColumn);
         ProjectedColumnExpression elemExpr = new ProjectedColumnExpression(elemColumn, columns, 0, elemColumn.getName().getString());
         ProjectedColumnExpression indexExpr = withOrdinality ? new ProjectedColumnExpression(indexColumn, columns, 1, indexColumn.getName().getString()) : null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
index 2788235..0856e79 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
@@ -25,6 +25,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PName;
@@ -43,7 +44,7 @@ public class ColumnExpressionTest {
         int scale = 5;
         PName colName = PNameFactory.newName("c1");
         PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PDecimal.INSTANCE, maxLen, scale,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -64,7 +65,7 @@ public class ColumnExpressionTest {
         int maxLen = 30;
         PName colName = PNameFactory.newName("c1");
         PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PBinary.INSTANCE, maxLen, null,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -85,7 +86,7 @@ public class ColumnExpressionTest {
         int scale = 5;
         PName colName = PNameFactory.newName("c1");
         PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PVarchar.INSTANCE, null, scale,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -105,7 +106,7 @@ public class ColumnExpressionTest {
     public void testSerializationWithNullScaleAndMaxLength() throws Exception {
         PName colName = PNameFactory.newName("c1");
         PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PDecimal.INSTANCE, null, null, true,
-                20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
+                20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 f49d291..45b61c1 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
@@ -90,7 +90,12 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Types;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -112,13 +117,20 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -138,6 +150,7 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil.ConnectionFactory;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
@@ -145,6 +158,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -218,26 +232,26 @@ public abstract class BaseTest {
                 "    CONSTRAINT pk PRIMARY KEY (organization_id, entity_id)\n" +
                 ") ");
         builder.put(TABLE_WITH_ARRAY, "create table "
-				+ TABLE_WITH_ARRAY
-				+ "   (organization_id char(15) not null, \n"
-				+ "    entity_id char(15) not null,\n"
-				+ "    a_string_array varchar(100) array[],\n"
-				+ "    b_string varchar(100),\n"
-				+ "    a_integer integer,\n"
-				+ "    a_date date,\n"
-				+ "    a_time time,\n"
-				+ "    a_timestamp timestamp,\n"
-				+ "    x_decimal decimal(31,10),\n"
-				+ "    x_long_array bigint array[],\n"
-				+ "    x_integer integer,\n"
-				+ "    a_byte_array tinyint array[],\n"
-				+ "    a_short smallint,\n"
-				+ "    a_float float,\n"
-				+ "    a_double_array double array[],\n"
-				+ "    a_unsigned_float unsigned_float,\n"
-				+ "    a_unsigned_double unsigned_double \n"
-				+ "    CONSTRAINT pk PRIMARY KEY (organization_id, entity_id)\n"
-				+ ")");
+                + TABLE_WITH_ARRAY
+                + "   (organization_id char(15) not null, \n"
+                + "    entity_id char(15) not null,\n"
+                + "    a_string_array varchar(100) array[],\n"
+                + "    b_string varchar(100),\n"
+                + "    a_integer integer,\n"
+                + "    a_date date,\n"
+                + "    a_time time,\n"
+                + "    a_timestamp timestamp,\n"
+                + "    x_decimal decimal(31,10),\n"
+                + "    x_long_array bigint array[],\n"
+                + "    x_integer integer,\n"
+                + "    a_byte_array tinyint array[],\n"
+                + "    a_short smallint,\n"
+                + "    a_float float,\n"
+                + "    a_double_array double array[],\n"
+                + "    a_unsigned_float unsigned_float,\n"
+                + "    a_unsigned_double unsigned_double \n"
+                + "    CONSTRAINT pk PRIMARY KEY (organization_id, entity_id)\n"
+                + ")");
         builder.put(BTABLE_NAME,"create table " + BTABLE_NAME +
                 "   (a_string varchar not null, \n" +
                 "    a_id char(3) not null,\n" +
@@ -388,7 +402,7 @@ public abstract class BaseTest {
     protected static String url;
     protected static PhoenixTestDriver driver;
     protected static boolean clusterInitialized = false;
-    private static HBaseTestingUtility utility;
+    protected static HBaseTestingUtility utility;
     protected static final Configuration config = HBaseConfiguration.create();
 
     private static class TearDownMiniClusterThreadFactory implements ThreadFactory {
@@ -720,9 +734,9 @@ public abstract class BaseTest {
             throw new IllegalStateException("Used up all unique names");
         }
         TABLE_COUNTER.incrementAndGet();
-        return "T" + Integer.toString(MAX_SUFFIX_VALUE + nextName).substring(1);
+        return "N" + Integer.toString(MAX_SUFFIX_VALUE + nextName).substring(1);
     }
-
+    
     private static AtomicInteger SEQ_NAME_SUFFIX = new AtomicInteger(0);
     private static final int MAX_SEQ_SUFFIX_VALUE = 1000000;
 
@@ -872,12 +886,18 @@ public abstract class BaseTest {
             // Make sure all tables and views have been dropped
             props.remove(CURRENT_SCN_ATTRIB);
             try (Connection seeLatestConn = DriverManager.getConnection(url, props)) {
-            	DatabaseMetaData dbmd = seeLatestConn.getMetaData();
-    	        ResultSet rs = dbmd.getTables(null, null, null, new String[]{PTableType.VIEW.toString(), PTableType.TABLE.toString()});
-    	        boolean hasTables = rs.next();
-    	        if (hasTables) {
-    	        	fail("The following tables are not deleted that should be:" + getTableNames(rs));
-    	        }
+                DatabaseMetaData dbmd = seeLatestConn.getMetaData();
+                ResultSet rs = dbmd.getTables(null, null, null, new String[]{PTableType.VIEW.toString(), PTableType.TABLE.toString()});
+                while (rs.next()) {
+                    String fullTableName = SchemaUtil.getEscapedTableName(
+                            rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM),
+                            rs.getString(PhoenixDatabaseMetaData.TABLE_NAME));
+                    try {
+                        PhoenixRuntime.getTable(conn, fullTableName);
+                        fail("The following tables are not deleted that should be:" + getTableNames(rs));
+                    } catch (TableNotFoundException e) {
+                    }
+                }
             }
         }
         finally {
@@ -926,12 +946,12 @@ public abstract class BaseTest {
     }
     
     private static String getTableNames(ResultSet rs) throws SQLException {
-    	StringBuilder buf = new StringBuilder();
-    	do {
-    		buf.append(" ");
-    		buf.append(SchemaUtil.getTableName(rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM), rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)));
-    	} while (rs.next());
-    	return buf.toString();
+        StringBuilder buf = new StringBuilder();
+        do {
+            buf.append(" ");
+            buf.append(SchemaUtil.getTableName(rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM), rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)));
+        } while (rs.next());
+        return buf.toString();
     }
 
     private static String getSchemaNames(ResultSet rs) throws SQLException {
@@ -1281,10 +1301,10 @@ public abstract class BaseTest {
     }
     
     private static String initEntityHistoryTableValues(String tableName, String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
-    	if (tableName == null) {
-    		tableName = generateUniqueName();
-    	}
-    	
+        if (tableName == null) {
+            tableName = generateUniqueName();
+        }
+        
         if (ts == null) {
             ensureTableCreated(url, tableName, ENTITY_HISTORY_TABLE_NAME, splits, null);
         } else {
@@ -1391,10 +1411,10 @@ public abstract class BaseTest {
     }
     
     protected static String initSaltedEntityHistoryTableValues(String tableName, String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
-    	if (tableName == null) {
-    		tableName = generateUniqueName();
-    	}
-    	
+        if (tableName == null) {
+            tableName = generateUniqueName();
+        }
+        
         if (ts == null) {
             ensureTableCreated(url, tableName, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, null);
         } else {
@@ -1613,42 +1633,42 @@ public abstract class BaseTest {
     }
     
     public static void upsertRows(Connection conn, String fullTableName, int numRows) throws SQLException {
-    	for (int i=1; i<=numRows; ++i) {
-	        upsertRow(conn, fullTableName, i, false);
-    	}
+        for (int i=1; i<=numRows; ++i) {
+            upsertRow(conn, fullTableName, i, false);
+        }
     }
 
     public static void upsertRow(Connection conn, String fullTableName, int index, boolean firstRowInBatch) throws SQLException {
-    	String upsert = "UPSERT INTO " + fullTableName
+        String upsert = "UPSERT INTO " + fullTableName
                 + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
-		PreparedStatement stmt = conn.prepareStatement(upsert);
-		stmt.setString(1, firstRowInBatch ? "firstRowInBatch_" : "" + "varchar"+index);
-		stmt.setString(2, "char"+index);
-		stmt.setInt(3, index);
-		stmt.setLong(4, index);
-		stmt.setBigDecimal(5, new BigDecimal(index));
-		Date date = DateUtil.parseDate("2015-01-01 00:00:00");
-		stmt.setDate(6, date);
-		stmt.setString(7, "varchar_a");
-		stmt.setString(8, "chara");
-		stmt.setInt(9, index+1);
-		stmt.setLong(10, index+1);
-		stmt.setBigDecimal(11, new BigDecimal(index+1));
-		stmt.setDate(12, date);
-		stmt.setString(13, "varchar_b");
-		stmt.setString(14, "charb");
-		stmt.setInt(15, index+2);
-		stmt.setLong(16, index+2);
-		stmt.setBigDecimal(17, new BigDecimal(index+2));
-		stmt.setDate(18, date);
-		stmt.executeUpdate();
-	}
+        PreparedStatement stmt = conn.prepareStatement(upsert);
+        stmt.setString(1, firstRowInBatch ? "firstRowInBatch_" : "" + "varchar"+index);
+        stmt.setString(2, "char"+index);
+        stmt.setInt(3, index);
+        stmt.setLong(4, index);
+        stmt.setBigDecimal(5, new BigDecimal(index));
+        Date date = DateUtil.parseDate("2015-01-01 00:00:00");
+        stmt.setDate(6, date);
+        stmt.setString(7, "varchar_a");
+        stmt.setString(8, "chara");
+        stmt.setInt(9, index+1);
+        stmt.setLong(10, index+1);
+        stmt.setBigDecimal(11, new BigDecimal(index+1));
+        stmt.setDate(12, date);
+        stmt.setString(13, "varchar_b");
+        stmt.setString(14, "charb");
+        stmt.setInt(15, index+2);
+        stmt.setLong(16, index+2);
+        stmt.setBigDecimal(17, new BigDecimal(index+2));
+        stmt.setDate(18, date);
+        stmt.executeUpdate();
+    }
 
     // Populate the test table with data.
     public static void populateTestTable(String fullTableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-        	upsertRows(conn, fullTableName, 3);
+            upsertRows(conn, fullTableName, 3);
             conn.commit();
         }
     }
@@ -1758,4 +1778,156 @@ public abstract class BaseTest {
         }
         phxConn.close();
     }
+    
+
+    /**
+     *  Split SYSTEM.CATALOG at the given split point 
+     */
+    protected static void splitRegion(byte[] splitPoint) throws SQLException, IOException, InterruptedException {
+        HBaseAdmin admin =
+                driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+        admin.split(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME, splitPoint);
+        // make sure the split finishes (there's no synchronous splitting before HBase 2.x)
+        admin.disableTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME);
+        admin.enableTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME);
+    }
+    
+    /**
+     * Returns true if the region contains atleast one of the metadata rows we are interested in
+     */
+    protected static boolean regionContainsMetadataRows(HRegionInfo regionInfo,
+            List<byte[]> metadataRowKeys) {
+        for (byte[] rowKey : metadataRowKeys) {
+            if (regionInfo.containsRow(rowKey)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    /**
+     * Splits SYSTEM.CATALOG into multiple regions based on the table or view names passed in.
+     * Metadata for each table or view is moved to a separate region,
+     * @param tenantToTableAndViewMap map from tenant to tables and views owned by the tenant
+     */
+    protected static void splitSystemCatalog(Map<String, List<String>> tenantToTableAndViewMap) throws Exception  {
+        List<byte[]> splitPoints = Lists.newArrayListWithExpectedSize(5);
+        // add the rows keys of the table or view metadata rows
+        Set<String> schemaNameSet=Sets.newHashSetWithExpectedSize(15);
+        for (Entry<String, List<String>> entrySet : tenantToTableAndViewMap.entrySet()) {
+            String tenantId = entrySet.getKey();
+            for (String fullName : entrySet.getValue()) {
+                String schemaName = SchemaUtil.getSchemaNameFromFullName(fullName);
+                // we don't allow SYSTEM.CATALOG to split within a schema, so to ensure each table
+                // or view is on a separate region they need to have a unique tenant and schema name
+                assertTrue("Schema names of tables/view must be unique ", schemaNameSet.add(tenantId+"."+schemaName));
+                String tableName = SchemaUtil.getTableNameFromFullName(fullName);
+                splitPoints.add(
+                    SchemaUtil.getTableKey(tenantId, "".equals(schemaName) ? null : schemaName, tableName));
+            }
+        }
+        Collections.sort(splitPoints, Bytes.BYTES_COMPARATOR);
+        
+        HBaseAdmin admin =
+                driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+        assertTrue("Needs at least two split points ", splitPoints.size() > 1);
+        assertTrue(
+            "Number of split points should be less than or equal to the number of region servers ",
+            splitPoints.size() <= NUM_SLAVES_BASE);
+        HBaseTestingUtility util = getUtility();
+        MiniHBaseCluster cluster = util.getHBaseCluster();
+        HMaster master = cluster.getMaster();
+        AssignmentManager am = master.getAssignmentManager();
+        // No need to split on the first splitPoint since the end key of region boundaries are exclusive
+        for (int i=1; i<splitPoints.size(); ++i) {
+            splitRegion(splitPoints.get(i));
+        }
+        HashMap<ServerName, List<HRegionInfo>> serverToRegionsList = Maps.newHashMapWithExpectedSize(NUM_SLAVES_BASE);
+        Deque<ServerName> availableRegionServers = new ArrayDeque<ServerName>(NUM_SLAVES_BASE);
+        for (int i=0; i<NUM_SLAVES_BASE; ++i) {
+            availableRegionServers.push(util.getHBaseCluster().getRegionServer(i).getServerName());
+        }
+        List<HRegionInfo> tableRegions =
+                admin.getTableRegions(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME);
+        for (HRegionInfo hRegionInfo : tableRegions) {
+            // filter on regions we are interested in
+            if (regionContainsMetadataRows(hRegionInfo, splitPoints)) {
+                ServerName serverName = am.getRegionStates().getRegionServerOfRegion(hRegionInfo);
+                if (!serverToRegionsList.containsKey(serverName)) {
+                    serverToRegionsList.put(serverName, new ArrayList<HRegionInfo>());
+                }
+                serverToRegionsList.get(serverName).add(hRegionInfo);
+                availableRegionServers.remove(serverName);
+                // Scan scan = new Scan();
+                // scan.setStartRow(hRegionInfo.getStartKey());
+                // scan.setStopRow(hRegionInfo.getEndKey());
+                // HTable primaryTable = new HTable(getUtility().getConfiguration(),
+                // PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME);
+                // ResultScanner resultScanner = primaryTable.getScanner(scan);
+                // for (Result result : resultScanner) {
+                // System.out.println(result);
+                // }
+            }
+        }
+        assertTrue("No region servers available to move regions on to ", !availableRegionServers.isEmpty());
+        for (Entry<ServerName, List<HRegionInfo>> entry : serverToRegionsList.entrySet()) {
+            List<HRegionInfo> regions = entry.getValue();
+            if (regions.size()>1) {
+                for (int i=1; i< regions.size(); ++i) {
+                    moveRegion(regions.get(i), entry.getKey(), availableRegionServers.pop());
+                }
+            }
+        }
+        
+        // verify each region of SYSTEM.CATALOG is on its own region server
+        tableRegions =
+                admin.getTableRegions(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME);
+        Set<ServerName> serverNames = Sets.newHashSet();
+        for (HRegionInfo hRegionInfo : tableRegions) {
+            // filter on regions we are interested in
+            if (regionContainsMetadataRows(hRegionInfo, splitPoints)) {
+                ServerName serverName = am.getRegionStates().getRegionServerOfRegion(hRegionInfo);
+                if (!serverNames.contains(serverName)) {
+                    serverNames.add(serverName);
+                }
+                else {
+                    fail("Multiple regions on "+serverName.getServerName());
+                }
+            }
+        }
+    }
+    
+    private static int getRegionServerIndex(MiniHBaseCluster cluster, ServerName serverName) {
+        // we have a small number of region servers, this should be fine for now.
+        List<RegionServerThread> servers = cluster.getRegionServerThreads();
+        for (int i = 0; i < servers.size(); i++) {
+            if (servers.get(i).getRegionServer().getServerName().equals(serverName)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+    
+    /**
+     * Ensures each region of SYSTEM.CATALOG is on a different region server
+     */
+    private static void moveRegion(HRegionInfo regionInfo, ServerName srcServerName, ServerName dstServerName) throws Exception  {
+        HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+        HBaseTestingUtility util = getUtility();
+        MiniHBaseCluster cluster = util.getHBaseCluster();
+        HMaster master = cluster.getMaster();
+        AssignmentManager am = master.getAssignmentManager();
+   
+        HRegionServer dstServer = cluster.getRegionServer(getRegionServerIndex(cluster, dstServerName));
+        HRegionServer srcServer = cluster.getRegionServer(getRegionServerIndex(cluster, srcServerName));
+        byte[] encodedRegionNameInBytes = regionInfo.getEncodedNameAsBytes();
+        admin.move(encodedRegionNameInBytes, Bytes.toBytes(dstServer.getServerName().getServerName()));
+        while (dstServer.getOnlineRegion(regionInfo.getRegionName()) == null
+                || dstServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameInBytes)
+                || srcServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameInBytes)
+                || master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+            // wait for the move to be finished
+            Thread.sleep(100);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 0443b77..485a21f 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
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -62,7 +63,26 @@ public class MetaDataUtilTest {
         assertFalse(MetaDataUtil.areClientAndServerCompatible(VersionUtil.encodeVersion(3,1,10), 3, 5));
     }
 
-  /**
+    @Test
+    public void testMutatingAPut() throws Exception {
+        String version = VersionInfo.getVersion();
+        KeyValueBuilder builder = KeyValueBuilder.get(version);
+        byte[] row = Bytes.toBytes("row");
+        byte[] family = PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+        byte[] qualifier = Bytes.toBytes("qual");
+        byte[] value = Bytes.toBytes("generic-value");
+        KeyValue kv = builder.buildPut(wrap(row), wrap(family), wrap(qualifier), wrap(value));
+        Put put = new Put(row);
+        KeyValueBuilder.addQuietly(put, builder, kv);
+        byte[] newValue = Bytes.toBytes("new-value");
+        Cell cell = put.get(family, qualifier).get(0);
+        assertEquals(Bytes.toString(value), Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+        MetaDataUtil.mutatePutValue(put, family, qualifier, newValue);
+        cell = put.get(family, qualifier).get(0);
+        assertEquals(Bytes.toString(newValue), Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+    }
+
+    /**
    * Ensure it supports {@link GenericKeyValueBuilder}
    * @throws Exception on failure
    */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 6920772..1683a13 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
@@ -132,7 +132,7 @@ public class TestUtil {
     private static final Log LOG = LogFactory.getLog(TestUtil.class);
     
     private static final Long ZERO = new Long(0);
-    public static final String DEFAULT_SCHEMA_NAME = "";
+    public static final String DEFAULT_SCHEMA_NAME = "S";
     public static final String DEFAULT_DATA_TABLE_NAME = "T";
     public static final String DEFAULT_INDEX_TABLE_NAME = "I";
     public static final String DEFAULT_DATA_TABLE_FULL_NAME = SchemaUtil.getTableName(DEFAULT_SCHEMA_NAME, "T");
@@ -722,6 +722,22 @@ public class TestUtil {
                 public String getExpressionStr() {
                     return null;
                 }
+
+                @Override
+                public long getTimestamp() {
+                    return HConstants.LATEST_TIMESTAMP;
+                }
+
+                @Override
+                public boolean isDerived() {
+                    return false;
+                }
+
+                @Override
+                public boolean isExcluded() {
+                    return false;
+                }
+
                 @Override
                 public boolean isRowTimestamp() {
                     return false;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 369522c..13d8f1a 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -50,6 +50,8 @@ enum MutationCode {
   AUTO_PARTITION_SEQUENCE_NOT_FOUND = 20;
   CANNOT_COERCE_AUTO_PARTITION_ID = 21;
   TOO_MANY_INDEXES = 22;
+  UNABLE_TO_CREATE_CHILD_LINK = 23;
+  UNABLE_TO_UPDATE_PARENT_TABLE = 24;
 };
 
 message SharedTableState {
@@ -83,6 +85,9 @@ message GetTableRequest {
   required int64 tableTimestamp = 4;
   required int64 clientTimestamp = 5;
   optional int32 clientVersion = 6;	
+  optional bool skipAddingParentColumns = 7;
+  optional bool skipAddingIndexes = 8;
+  optional PTable lockedAncestorTable = 9;
 }
 
 message GetFunctionsRequest {
@@ -125,6 +130,7 @@ message DropTableRequest {
   required string tableType = 2;
   optional bool cascade = 3;
   optional int32 clientVersion = 4;
+  optional bool skipAddingParentColumns = 5;
 }
 
 message DropSchemaRequest {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index 16381dd..bc868bc 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -35,12 +35,12 @@ enum PTableType {
 message PColumn {
   required bytes columnNameBytes = 1;
   optional bytes familyNameBytes = 2;
-  required string dataType = 3;
+  optional string dataType = 3;
   optional int32 maxLength = 4;
   optional int32 scale = 5;
   required bool nullable = 6;
   required int32 position = 7;
-  required int32 sortOrder = 8;
+  optional int32 sortOrder = 8;
   optional int32 arraySize = 9;
   optional bytes viewConstant = 10;
   optional bool viewReferenced = 11;
@@ -48,6 +48,8 @@ message PColumn {
   optional bool isRowTimestamp = 13;
   optional bool isDynamic = 14;
   optional bytes columnQualifierBytes = 15;
+  optional int64 timestamp = 16;
+  optional bool derived = 17 [default = false];
 }
 
 message PTableStats {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7e74e63..4412c15 100644
--- a/pom.xml
+++ b/pom.xml
@@ -320,6 +320,24 @@
                  <goal>verify</goal>
               </goals>
             </execution>
+<execution>
+              <id>SplitSystemCatalogTests</id>
+              <configuration>
+                 <encoding>UTF-8</encoding>
+                 <forkCount>${numForkedIT}</forkCount>
+                 <runOrder>alphabetical</runOrder>
+                 <reuseForks>false</reuseForks>
+                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
+                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                 <groups>org.apache.phoenix.end2end.SplitSystemCatalogTests</groups>
+                 <shutdown>kill</shutdown>
+              </configuration>
+              <goals>
+                 <goal>integration-test</goal>
+                 <goal>verify</goal>
+              </goals>
+            </execution>
           </executions>
         </plugin>
         <plugin>


[47/50] [abbrv] phoenix git commit: PHOENIX-4903 Use same hash cache RPC message across all calls

Posted by ja...@apache.org.
PHOENIX-4903 Use same hash cache RPC message across all calls

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: d8a78e19d9c1984d5ae3d8bdfe55fcec5a9b983f
Parents: 4d8c5d4
Author: Marcell Ortutay <ma...@gmail.com>
Authored: Sat Sep 15 21:13:54 2018 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Tue Sep 18 14:30:10 2018 -0400

----------------------------------------------------------------------
 .../apache/phoenix/cache/ServerCacheClient.java | 61 ++++++++++----------
 1 file changed, 32 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d8a78e19/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 011a6f8..93d16f5 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
@@ -486,6 +486,37 @@ public class ServerCacheClient {
             throws Exception {
         byte[] keyInRegion = getKeyInRegion(key);
         final Map<byte[], AddServerCacheResponse> results;
+
+        AddServerCacheRequest.Builder builder = AddServerCacheRequest.newBuilder();
+        final byte[] tenantIdBytes;
+        if (cacheUsingTable.isMultiTenant()) {
+            try {
+                tenantIdBytes = connection.getTenantId() == null ? null
+                        : ScanUtil.getTenantIdBytes(cacheUsingTable.getRowKeySchema(),
+                        cacheUsingTable.getBucketNum() != null, connection.getTenantId(),
+                        cacheUsingTable.getViewIndexId() != null);
+            } catch (SQLException e) {
+                throw new IOException(e);
+            }
+        } else {
+            tenantIdBytes = connection.getTenantId() == null ? null
+                    : connection.getTenantId().getBytes();
+        }
+        if (tenantIdBytes != null) {
+            builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+        }
+        builder.setCacheId(ByteStringer.wrap(cacheId));
+        builder.setUsePersistentCache(usePersistentCache);
+        builder.setCachePtr(org.apache.phoenix.protobuf.ProtobufUtil.toProto(cachePtr));
+        builder.setHasProtoBufIndexMaintainer(true);
+        ServerCacheFactoryProtos.ServerCacheFactory.Builder svrCacheFactoryBuider = ServerCacheFactoryProtos.ServerCacheFactory
+                .newBuilder();
+        svrCacheFactoryBuider.setClassName(cacheFactory.getClass().getName());
+        builder.setCacheFactory(svrCacheFactoryBuider.build());
+        builder.setTxState(ByteStringer.wrap(txState));
+        builder.setClientVersion(MetaDataProtocol.PHOENIX_VERSION);
+        final AddServerCacheRequest request = builder.build();
+
         try {
             results = htable.coprocessorService(ServerCachingService.class, keyInRegion, keyInRegion,
                     new Batch.Call<ServerCachingService, AddServerCacheResponse>() {
@@ -493,35 +524,7 @@ public class ServerCacheClient {
                         public AddServerCacheResponse call(ServerCachingService instance) throws IOException {
                             ServerRpcController controller = new ServerRpcController();
                             BlockingRpcCallback<AddServerCacheResponse> rpcCallback = new BlockingRpcCallback<AddServerCacheResponse>();
-                            AddServerCacheRequest.Builder builder = AddServerCacheRequest.newBuilder();
-                            final byte[] tenantIdBytes;
-                            if (cacheUsingTable.isMultiTenant()) {
-                                try {
-                                    tenantIdBytes = connection.getTenantId() == null ? null
-                                            : ScanUtil.getTenantIdBytes(cacheUsingTable.getRowKeySchema(),
-                                                    cacheUsingTable.getBucketNum() != null, connection.getTenantId(),
-                                                    cacheUsingTable.getViewIndexId() != null);
-                                } catch (SQLException e) {
-                                    throw new IOException(e);
-                                }
-                            } else {
-                                tenantIdBytes = connection.getTenantId() == null ? null
-                                        : connection.getTenantId().getBytes();
-                            }
-                            if (tenantIdBytes != null) {
-                                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                            }
-                            builder.setCacheId(ByteStringer.wrap(cacheId));
-                            builder.setUsePersistentCache(usePersistentCache);
-                            builder.setCachePtr(org.apache.phoenix.protobuf.ProtobufUtil.toProto(cachePtr));
-                            builder.setHasProtoBufIndexMaintainer(true);
-                            ServerCacheFactoryProtos.ServerCacheFactory.Builder svrCacheFactoryBuider = ServerCacheFactoryProtos.ServerCacheFactory
-                                    .newBuilder();
-                            svrCacheFactoryBuider.setClassName(cacheFactory.getClass().getName());
-                            builder.setCacheFactory(svrCacheFactoryBuider.build());
-                            builder.setTxState(ByteStringer.wrap(txState));
-                            builder.setClientVersion(MetaDataProtocol.PHOENIX_VERSION);
-                            instance.addServerCache(controller, builder.build(), rpcCallback);
+                            instance.addServerCache(controller, request, rpcCallback);
                             if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
                             return rpcCallback.get();
                         }


[03/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index 45aca98..a267629 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.schema;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.query.QueryConstants;
@@ -42,36 +43,63 @@ public class PColumnImpl implements PColumn {
     private boolean isRowTimestamp;
     private boolean isDynamic;
     private byte[] columnQualifierBytes;
-    
+    private boolean derived;
+    private long timestamp;
+
     public PColumnImpl() {
     }
 
-    public PColumnImpl(PName name,
-                       PName familyName,
-                       PDataType dataType,
-                       Integer maxLength,
-                       Integer scale,
-                       boolean nullable,
-                       int position,
-                       SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic, byte[] columnQualifierBytes) {
-        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes);
+    public PColumnImpl(PColumn column, int position) {
+        this(column, column.isDerived(), position);
     }
 
-    public PColumnImpl(PColumn column, int position) {
+    public PColumnImpl(PColumn column, byte[] viewConstant, boolean isViewReferenced) {
+        this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
+                column.getScale(), column.isNullable(), column.getPosition(), column.getSortOrder(), column.getArraySize(), viewConstant, isViewReferenced, column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes(),
+                column.getTimestamp(), column.isDerived());
+    }
+
+    public PColumnImpl(PColumn column, boolean derivedColumn, int position) {
+        this(column, derivedColumn, position, column.getViewConstant());
+    }
+
+    public PColumnImpl(PColumn column, boolean derivedColumn, int position, byte[] viewConstant) {
         this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
-                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes());
+            column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), viewConstant, column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes(),
+            column.getTimestamp(), derivedColumn);
+    }
+    
+    public PColumnImpl(PName name, PName familyName, PDataType dataType, Integer maxLength, Integer scale, boolean nullable,
+        int position, SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic,
+        byte[] columnQualifierBytes, long timestamp) {
+        this(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes, timestamp, false);
+    }
+
+    public PColumnImpl(PName name, PName familyName, PDataType dataType, Integer maxLength, Integer scale, boolean nullable,
+        int position, SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic,
+        byte[] columnQualifierBytes, long timestamp, boolean derived) {
+        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes, timestamp, derived);
+    }
+
+    private PColumnImpl(PName familyName, PName columnName, Long timestamp) {
+        this.familyName = familyName;
+        this.name = columnName;
+        this.derived = true;
+        if (timestamp!=null) {
+            this.timestamp = timestamp;
+        }
     }
 
-    private void init(PName name,
-            PName familyName,
-            PDataType dataType,
-            Integer maxLength,
-            Integer scale,
-            boolean nullable,
-            int position,
-            SortOrder sortOrder,
-            Integer arrSize,
-            byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic, byte[] columnQualifierBytes) {
+    // a derived column has null type
+    public static PColumnImpl createExcludedColumn(PName familyName, PName columnName, Long timestamp) {
+        return new PColumnImpl(familyName, columnName, timestamp);
+    }
+    
+    private void init(PName name, PName familyName, PDataType dataType, Integer maxLength,
+            Integer scale, boolean nullable, int position, SortOrder sortOrder, Integer arrSize,
+            byte[] viewConstant, boolean isViewReferenced, String expressionStr,
+            boolean isRowTimestamp, boolean isDynamic, byte[] columnQualifierBytes, long timestamp,
+            boolean derived) {
     	Preconditions.checkNotNull(sortOrder);
         this.dataType = dataType;
         if (familyName == null) {
@@ -97,6 +125,8 @@ public class PColumnImpl implements PColumn {
         this.isRowTimestamp = isRowTimestamp;
         this.isDynamic = isDynamic;
         this.columnQualifierBytes = columnQualifierBytes;
+        this.timestamp = timestamp;
+        this.derived = derived;
     }
 
     @Override
@@ -137,6 +167,16 @@ public class PColumnImpl implements PColumn {
     }
 
     @Override
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    @Override
+    public boolean isExcluded() {
+        return dataType == null;
+    }
+
+    @Override
     public boolean isNullable() {
         return nullable;
     }
@@ -226,7 +266,7 @@ public class PColumnImpl implements PColumn {
         if (column.hasFamilyNameBytes()) {
             familyName = PNameFactory.newName(column.getFamilyNameBytes().toByteArray());
         }
-        PDataType dataType = PDataType.fromSqlTypeName(column.getDataType());
+        PDataType dataType = column.hasDataType() ? PDataType.fromSqlTypeName(column.getDataType()) : null;
         Integer maxLength = null;
         if (column.hasMaxLength()) {
             maxLength = column.getMaxLength();
@@ -263,8 +303,17 @@ public class PColumnImpl implements PColumn {
         if (column.hasColumnQualifierBytes()) {
             columnQualifierBytes = column.getColumnQualifierBytes().toByteArray();
         }
+        long timestamp = HConstants.LATEST_TIMESTAMP;
+        if (column.hasTimestamp()) {
+            timestamp = column.getTimestamp();
+        }
+        boolean derived = false;
+        if (column.hasDerived()) {
+            derived = column.getDerived();
+        }
         return new PColumnImpl(columnName, familyName, dataType, maxLength, scale, nullable, position, sortOrder,
-                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes);
+                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes,
+            timestamp, derived);
     }
 
     public static PTableProtos.PColumn toProto(PColumn column) {
@@ -273,7 +322,9 @@ public class PColumnImpl implements PColumn {
         if (column.getFamilyName() != null) {
             builder.setFamilyNameBytes(ByteStringer.wrap(column.getFamilyName().getBytes()));
         }
-        builder.setDataType(column.getDataType().getSqlTypeName());
+        if (column.getDataType()!=null) {
+            builder.setDataType(column.getDataType().getSqlTypeName());
+        }
         if (column.getMaxLength() != null) {
             builder.setMaxLength(column.getMaxLength());
         }
@@ -282,7 +333,9 @@ public class PColumnImpl implements PColumn {
         }
         builder.setNullable(column.isNullable());
         builder.setPosition(column.getPosition());
-        builder.setSortOrder(column.getSortOrder().getSystemValue());
+        if (column.getSortOrder()!=null) {
+            builder.setSortOrder(column.getSortOrder().getSystemValue());
+        }
         if (column.getArraySize() != null) {
             builder.setArraySize(column.getArraySize());
         }
@@ -298,6 +351,14 @@ public class PColumnImpl implements PColumn {
         if (column.getColumnQualifierBytes() != null) {
             builder.setColumnQualifierBytes(ByteStringer.wrap(column.getColumnQualifierBytes()));
         }
+        if (column.getTimestamp() != HConstants.LATEST_TIMESTAMP) {
+            builder.setTimestamp(column.getTimestamp());
+        }
+        builder.setDerived(column.isDerived());
         return builder.build();
     }
+
+    public boolean isDerived() {
+        return derived;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 17f929f..598cc79 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -214,7 +214,8 @@ public class PMetaDataImpl implements PMetaData {
             // Update position of columns that follow removed column
             for (int i = position+1; i < oldColumns.size(); i++) {
                 PColumn oldColumn = oldColumns.get(i);
-                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getColumnQualifierBytes());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getColumnQualifierBytes(),
+                    oldColumn.getTimestamp());
                 columns.add(newColumn);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 af78612..bb81d76 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
@@ -151,15 +151,26 @@ public interface PTable extends PMetaDataEntity {
         PARENT_TABLE((byte)3),
         /**
          * Link from a parent table to its child view
+         * (these are stored in SYSTEM.CHILD_LINK for scalability)
          */
-        CHILD_TABLE((byte)4);
+        CHILD_TABLE((byte)4),
+        /**
+         * Link for an excluded (dropped) column
+         */
+        EXCLUDED_COLUMN((byte)5),
+        /**
+         * Link from an index on a view to its parent table
+         */
+        VIEW_INDEX_PARENT_TABLE((byte)6);
 
         private final byte[] byteValue;
         private final byte serializedValue;
+        private final byte[] serializedByteArrayValue;
 
         LinkType(byte serializedValue) {
             this.serializedValue = serializedValue;
             this.byteValue = Bytes.toBytes(this.name());
+            this.serializedByteArrayValue = new byte[] { serializedValue };
         }
 
         public byte[] getBytes() {
@@ -170,6 +181,10 @@ public interface PTable extends PMetaDataEntity {
             return this.serializedValue;
         }
 
+        public byte[] getSerializedValueAsByteArray() {
+            return serializedByteArrayValue;
+        }
+
         public static LinkType fromSerializedValue(byte serializedValue) {
             if (serializedValue < 1 || serializedValue > LinkType.values().length) {
                 return null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 082a58b..d575902 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
@@ -28,6 +28,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -258,125 +259,81 @@ public class PTableImpl implements PTable {
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes) throws SQLException {
-        return makePTable(table, timeStamp, indexes, table.getParentSchemaName(), table.getViewStatement());
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), timeStamp,
+                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), indexes);
     }
 
-    public static PTable makePTable(PTable index, PName indexName, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
-        return Objects.equal(viewStatement, index.getViewStatement()) ? index : makePTable(index, indexName, index.getTimeStamp(), Lists.newArrayList(index.getPhysicalName()), index.getIndexes(), viewStatement, updateCacheFrequency, tenantId);
+    public static PTable makePTable(PTable index, PName indexName, String viewStatement, long updateCacheFrequency,
+            PName tenantId) throws SQLException {
+        return Objects.equal(viewStatement, index.getViewStatement()) ? index
+                : new PTableImpl(index, index.rowKeyOrderOptimizable(), index.getIndexState(), index.getTimeStamp(),
+                        index.getSequenceNumber(), index.getColumns(), index.getDefaultFamilyName(), index.getType(),
+                        index.getBaseColumnCount(), index.getSchemaName(), indexName,
+                        viewStatement, updateCacheFrequency, tenantId,
+                        index.getIndexes());
     }
     
-    public static PTableImpl makePTable(PTable table, PName tableName, long timeStamp, List<PName> physicalNames, List<PTable> indexes, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
-            return new PTableImpl(
-                    tenantId, table.getSchemaName(), tableName, table.getType(), table.getIndexState(), timeStamp,
-                    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.getTransactionProvider(), updateCacheFrequency,
-                    table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
-        }
-
-    public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes, PName parentSchemaName, String viewStatement) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
-                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.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, Collection<PColumn> columns) throws SQLException {
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), columns, table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns) throws SQLException {
+    /**
+     * Used to create a PTable for views or view indexes, the basePTable is for attributes we inherit from the physical table
+     */
+    public static PTableImpl makePTable(PTable view, PTable baseTable, Collection<PColumn> columns, long timestamp, int baseTableColumnCount) throws SQLException {
+        // if a TableProperty is not valid on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is mutable on a view we use the value set on the view 
         return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                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.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+            view.getTenantId(), view.getSchemaName(), view.getTableName(), view.getType(), view.getIndexState(), timestamp,
+            view.getSequenceNumber(), view.getPKName(), view.getBucketNum(), columns, view.getParentSchemaName(), view.getParentTableName(),
+            view.getIndexes(), baseTable.isImmutableRows(), view.getPhysicalNames(), view.getDefaultFamilyName(), view.getViewStatement(),
+            baseTable.isWALDisabled(), baseTable.isMultiTenant(), baseTable.getStoreNulls(), view.getViewType(), view.getViewIndexId(), view.getIndexType(),
+            baseTableColumnCount, view.rowKeyOrderOptimizable(), baseTable.getTransactionProvider(), view.getUpdateCacheFrequency(),
+            view.getIndexDisableTimestamp(), view.isNamespaceMapped(), baseTable.getAutoPartitionSeqName(), baseTable.isAppendOnlySchema(),
+            baseTable.getImmutableStorageScheme(), baseTable.getEncodingScheme(), view.getEncodedCQCounter(), view.useStatsForParallelization());
     }
     
     public static PTableImpl makePTable(PTable table, PTableType type, Collection<PColumn> columns) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), type, table.getIndexState(), table.getTimeStamp(),
-                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.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), columns, table.getDefaultFamilyName(), type,
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns, PName defaultFamily) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                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.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, Collection<PColumn> columns, PName defaultFamily)
+            throws SQLException {
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), columns, defaultFamily, table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns) 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(),
-                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.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)
+            throws SQLException {
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), timeStamp,
+                sequenceNumber, columns, table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows) 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(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                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, 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(), transactionProvider, updateCacheFrequency, table.getIndexDisableTimestamp(), 
-                isNamespaceMapped, table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
-    }
-    
     public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), state, table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table),
-                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.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), state, table.getTimeStamp(),
+                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
     public static PTableImpl makePTable(PTable table, boolean rowKeyOrderOptimizable) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table),
-                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.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) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table),
-                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.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
-                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return new PTableImpl(table, rowKeyOrderOptimizable, table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -384,13 +341,18 @@ 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, 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 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, transactionProvider,
-                updateCacheFrequency,indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+                updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName,
+                isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter,
+                useStatsForParallelization);
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -398,16 +360,31 @@ 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, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
-            int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
+            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)
-            throws SQLException {
+            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, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, 
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+                indexType, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency,
+                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme,
+                qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+    }
+
+    private PTableImpl(PTable table, boolean rowKeyOrderOptimizable, PIndexState state, long timeStamp,
+            long sequenceNumber, Collection<PColumn> columns, PName defaultFamily, PTableType type,
+            int baseTableColumnCount, PName schemaName, PName tableName, String viewStatement,
+            long updateCacheFrequency, PName tenantId, List<PTable> indexes) throws SQLException {
+        init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, table.getPKName(),
+                table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), indexes,
+                table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, viewStatement, table.isWALDisabled(),
+                table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
+                table.getIndexType(), baseTableColumnCount, rowKeyOrderOptimizable, table.getTransactionProvider(),
+                updateCacheFrequency, table.getIndexDisableTimestamp(), table.isNamespaceMapped(),
+                table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(),
+                table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
@@ -513,8 +490,23 @@ public class PTableImpl implements PTable {
             allColumns = new PColumn[columns.size()];
             pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
         }
-        for (PColumn column : columns) {
-            allColumns[column.getPosition()] = column;
+        // Must do this as with the new method of storing diffs, we just care about ordinal position
+        // relative order and not the true ordinal value itself.
+        List<PColumn> sortedColumns = Lists.newArrayList(columns);
+        Collections.sort(sortedColumns, new Comparator<PColumn>() {
+            @Override
+            public int compare(PColumn o1, PColumn o2) {
+                return Integer.valueOf(o1.getPosition()).compareTo(o2.getPosition());
+            }
+        });
+
+        int position = 0;
+        if (bucketNum != null) {
+            position = 1;
+        }
+        for (PColumn column : sortedColumns) {
+            allColumns[position] = column;
+            position++;
             PName familyName = column.getFamilyName();
             if (familyName == null) {
                 ++numPKColumns;
@@ -566,10 +558,10 @@ public class PTableImpl implements PTable {
                                 || column.getDataType() == PDouble.INSTANCE 
                                 || column.getDataType() == PBinary.INSTANCE) )
                         || (column.getSortOrder() == SortOrder.ASC && column.getDataType() == PBinary.INSTANCE && column.getMaxLength() != null && column.getMaxLength() > 1);
-            	pkColumns.add(column);
-            	if (column.isRowTimestamp()) {
-            	    rowTimestampCol = column;
-            	}
+                pkColumns.add(column);
+                if (column.isRowTimestamp()) {
+                    rowTimestampCol = column;
+                }
             }
             if (familyName == null) {
                 estimatedSize += column.getEstimatedSize(); // PK columns
@@ -752,7 +744,7 @@ public class PTableImpl implements PTable {
                 if (byteValue.length == 0 && !column.isNullable()) {
                     throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
                 }
-                Integer	maxLength = column.getMaxLength();
+                Integer    maxLength = column.getMaxLength();
                 Integer scale = column.getScale();
                 key.set(byteValue);
                 if (!type.isSizeCompatible(key, null, type, sortOrder, null, null, maxLength, scale)) {
@@ -933,10 +925,10 @@ public class PTableImpl implements PTable {
                         }
                         Expression[] colValues = EncodedColumnsUtil.createColumnExpressionArray(maxEncodedColumnQualifier);
                         for (PColumn column : columns) {
-                        	if (columnToValueMap.containsKey(column)) {
-                        	    int colIndex = qualifierEncodingScheme.decode(column.getColumnQualifierBytes())-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
-                        	    colValues[colIndex] = new LiteralExpression(columnToValueMap.get(column));
-                        	}
+                            if (columnToValueMap.containsKey(column)) {
+                                int colIndex = qualifierEncodingScheme.decode(column.getColumnQualifierBytes())-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
+                                colValues[colIndex] = new LiteralExpression(columnToValueMap.get(column));
+                            }
                         }
                         
                         List<Expression> children = Arrays.asList(colValues);
@@ -1010,7 +1002,7 @@ public class PTableImpl implements PTable {
                             .getFamilyName().getBytesPtr(), qualifierPtr, ts));
             } else {
                 ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue);
-                Integer	maxLength = column.getMaxLength();
+                Integer    maxLength = column.getMaxLength();
                 Integer scale = column.getScale();
                 SortOrder sortOrder = column.getSortOrder();
                 if (!type.isSizeCompatible(ptr, null, type, sortOrder, null, null, maxLength, scale)) {
@@ -1192,7 +1184,7 @@ public class PTableImpl implements PTable {
 
     @Override
     public List<PName> getPhysicalNames() {
-        return physicalNames;
+        return !physicalNames.isEmpty() ? physicalNames : Lists.newArrayList(getPhysicalName());
     }
 
     @Override
@@ -1230,6 +1222,8 @@ public class PTableImpl implements PTable {
      * @param table
      */
     public static PTable createFromProto(PTableProtos.PTable table) {
+        if (table==null)
+            return null;
         PName tenantId = null;
         if(table.hasTenantId()){
             tenantId = PNameFactory.newName(table.getTenantId().toByteArray());
@@ -1339,15 +1333,15 @@ public class PTableImpl implements PTable {
         }
         EncodedCQCounter encodedColumnQualifierCounter = null;
         if ((!EncodedColumnsUtil.usesEncodedColumnNames(qualifierEncodingScheme) || tableType == PTableType.VIEW)) {
-        	encodedColumnQualifierCounter = PTable.EncodedCQCounter.NULL_COUNTER;
+            encodedColumnQualifierCounter = PTable.EncodedCQCounter.NULL_COUNTER;
         }
         else {
-        	encodedColumnQualifierCounter = new EncodedCQCounter();
-        	if (table.getEncodedCQCountersList() != null) {
-        		for (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter cqCounterFromProto : table.getEncodedCQCountersList()) {
-        			encodedColumnQualifierCounter.setValue(cqCounterFromProto.getColFamily(), cqCounterFromProto.getCounter());
-        		}
-        	}
+            encodedColumnQualifierCounter = new EncodedCQCounter();
+            if (table.getEncodedCQCountersList() != null) {
+                for (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter cqCounterFromProto : table.getEncodedCQCountersList()) {
+                    encodedColumnQualifierCounter.setValue(cqCounterFromProto.getColFamily(), cqCounterFromProto.getCounter());
+                }
+            }
         }
         Boolean useStatsForParallelization = null;
         if (table.hasUseStatsForParallelization()) {
@@ -1376,15 +1370,15 @@ public class PTableImpl implements PTable {
       builder.setTableNameBytes(ByteStringer.wrap(table.getTableName().getBytes()));
       builder.setTableType(ProtobufUtil.toPTableTypeProto(table.getType()));
       if (table.getType() == PTableType.INDEX) {
-    	if(table.getIndexState() != null) {
-    	  builder.setIndexState(table.getIndexState().getSerializedValue());
-    	}
-    	if(table.getViewIndexId() != null) {
-    	  builder.setViewIndexId(table.getViewIndexId());
-    	}
-    	if(table.getIndexType() != null) {
-    	    builder.setIndexType(ByteStringer.wrap(new byte[]{table.getIndexType().getSerializedValue()}));
-    	}
+        if(table.getIndexState() != null) {
+          builder.setIndexState(table.getIndexState().getSerializedValue());
+        }
+        if(table.getViewIndexId() != null) {
+          builder.setViewIndexId(table.getViewIndexId());
+        }
+        if(table.getIndexType() != null) {
+            builder.setIndexType(ByteStringer.wrap(new byte[]{table.getIndexType().getSerializedValue()}));
+        }
       }
       builder.setSequenceNumber(table.getSequenceNumber());
       builder.setTimeStamp(table.getTimeStamp());
@@ -1403,10 +1397,9 @@ public class PTableImpl implements PTable {
       List<PColumn> columns = table.getColumns();
       int columnSize = columns.size();
       for (int i = offset; i < columnSize; i++) {
-        PColumn column = columns.get(i);
-        builder.addColumns(PColumnImpl.toProto(column));
+          PColumn column = columns.get(i);
+          builder.addColumns(PColumnImpl.toProto(column));
       }
-
       List<PTable> indexes = table.getIndexes();
       for (PTable curIndex : indexes) {
         builder.addIndexes(toProto(curIndex));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
index 017c75d..4b44595 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
@@ -45,14 +45,14 @@ public class PTableKey {
     
     @Override
     public String toString() {
-        return name + (tenantId == null ? "" : " for " + tenantId.getString());
+        return name + ((tenantId == null || tenantId.getBytes().length==0) ? "" : " for " + tenantId.getString());
     }
     
     @Override
     public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + ((tenantId == null) ? 0 : tenantId.hashCode());
+        result = prime * result + ((tenantId == null || tenantId.getBytes().length==0) ? 0 : tenantId.hashCode());
         result = prime * result + name.hashCode();
         return result;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
new file mode 100644
index 0000000..27446e0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
@@ -0,0 +1,30 @@
+package org.apache.phoenix.schema;
+
+import org.apache.phoenix.coprocessor.TableInfo;
+
+public class ParentTableNotFoundException extends TableNotFoundException {
+    private static final long serialVersionUID = 1L;
+    private final byte[] parentTenantId;
+    private final byte[] parentSchemaName;
+    private final byte[] parentTableName;
+
+    public ParentTableNotFoundException(TableInfo parentTableInfo, String tableName) {
+        super(tableName);
+        this.parentTenantId = parentTableInfo.getTenantId();
+        this.parentSchemaName = parentTableInfo.getSchemaName();
+        this.parentTableName = parentTableInfo.getTableName();
+    }
+
+    public byte[] getParentTenantId() {
+        return parentTenantId;
+    }
+
+    public byte[] getParentSchemaName() {
+        return parentSchemaName;
+    }
+
+    public byte[] getParentTableName() {
+        return parentTableName;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
index 23cfd1b..7bbd620 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.schema;
 
 import java.util.List;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.query.KeyRange;
@@ -38,7 +39,8 @@ public class SaltingUtil {
     public static final String SALTING_COLUMN_NAME = "_SALT";
     public static final String SALTED_ROW_KEY_NAME = "_SALTED_KEY";
     public static final PColumnImpl SALTING_COLUMN = new PColumnImpl(
-            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null);
+            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null,
+        HConstants.LATEST_TIMESTAMP);
     public static final RowKeySchema VAR_BINARY_SALTED_SCHEMA = new RowKeySchemaBuilder(2)
         .addField(SALTING_COLUMN, false, SortOrder.getDefault())
         .addField(SchemaUtil.VAR_BINARY_DATUM, false, SortOrder.getDefault()).build();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 3d473c4..3d2e84e 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
@@ -152,7 +152,7 @@ public enum TableProperty {
             return table.isAppendOnlySchema();
         }
     },
-    GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false, false) {
+    GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false) {
         @Override
         public Object getValue(Object value) {
             return value == null ? null : ((Number) value).longValue();
@@ -231,36 +231,26 @@ public enum TableProperty {
     private final SQLExceptionCode mutatingImmutablePropException;
     private final boolean isValidOnView;
     private final boolean isMutableOnView;
-    private final boolean propagateToViews;
 
     private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView) {
-        this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true);
-    }
-
-    private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView, boolean propagateToViews) {
-        this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, propagateToViews);
+        this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView);
     }
 
     private TableProperty(String propertyName, SQLExceptionCode colFamilySpecifiedException, boolean isMutable, boolean isValidOnView, boolean isMutableOnView) {
-        this(propertyName, colFamilySpecifiedException, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true);
+        this(propertyName, colFamilySpecifiedException, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView);
     }
 
     private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView, SQLExceptionCode isMutatingException) {
-        this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, isMutatingException, isValidOnView, isMutableOnView, true);
+        this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, isMutatingException, isValidOnView, isMutableOnView);
     }
 
     private TableProperty(String propertyName, SQLExceptionCode colFamSpecifiedException, boolean isMutable, SQLExceptionCode mutatingException, boolean isValidOnView, boolean isMutableOnView) {
-        this(propertyName, colFamSpecifiedException, isMutable, mutatingException, isValidOnView, isMutableOnView, true);
-    }
-
-    private TableProperty(String propertyName, SQLExceptionCode colFamSpecifiedException, boolean isMutable, SQLExceptionCode mutatingException, boolean isValidOnView, boolean isMutableOnView, boolean propagateToViews) {
         this.propertyName = propertyName;
         this.colFamSpecifiedException = colFamSpecifiedException;
         this.isMutable = isMutable;
         this.mutatingImmutablePropException = mutatingException;
         this.isValidOnView = isValidOnView;
         this.isMutableOnView = isMutableOnView;
-        this.propagateToViews = propagateToViews;
     }
 
     public static boolean isPhoenixTableProperty(String property) {
@@ -326,10 +316,6 @@ public enum TableProperty {
         return isMutableOnView;
     }
 
-    public boolean propagateToViews() {
-        return propagateToViews;
-    }
-
     abstract public Object getPTableValue(PTable table);
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 3fe5438..b63b053 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -100,7 +100,10 @@ import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PIndexState;
+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;
@@ -139,7 +142,7 @@ public class IndexUtil {
     
     // Since we cannot have nullable fixed length in a row key
     // we need to translate to variable length. The verification that we have a valid index
-    // row key was already done, so here we just need to covert from one built-in type to
+    // row key was already done, so here we just need to convert from one built-in type to
     // another.
     public static PDataType getIndexColumnDataType(boolean isNullable, PDataType dataType) {
         if (dataType == null || !isNullable || !dataType.isFixedWidth()) {
@@ -194,6 +197,17 @@ public class IndexUtil {
         String dataColumnFamilyName = SchemaUtil.isPKColumn(dataColumn) ? null : dataColumn.getFamilyName().getString();
         return getIndexColumnName(dataColumnFamilyName, dataColumn.getName().getString());
     }
+    
+	public static PColumn getIndexPKColumn(int position, PColumn dataColumn) {
+		assert (SchemaUtil.isPKColumn(dataColumn));
+		PName indexColumnName = PNameFactory.newName(getIndexColumnName(null, dataColumn.getName().getString()));
+		PColumn column = new PColumnImpl(indexColumnName, null, dataColumn.getDataType(), dataColumn.getMaxLength(),
+				dataColumn.getScale(), dataColumn.isNullable(), position, dataColumn.getSortOrder(),
+				dataColumn.getArraySize(), null, false, dataColumn.getExpressionStr(), dataColumn.isRowTimestamp(), false,
+				// TODO set the columnQualifierBytes correctly
+				/*columnQualifierBytes*/null, HConstants.LATEST_TIMESTAMP); 
+		return column;
+	}
 
     public static String getLocalIndexColumnFamily(String dataColumnFamilyName) {
         return dataColumnFamilyName == null ? null

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 1dda818..f7b7b4a 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
@@ -24,7 +24,9 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
+import java.util.NavigableMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -34,10 +36,12 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -61,6 +65,7 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SortOrder;
@@ -68,12 +73,15 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableProperty;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PSmallint;
 import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 import com.google.protobuf.ServiceException;
 
 
@@ -176,28 +184,104 @@ public class MetaDataUtil {
         return version;
     }
     
+    public static byte[] getTenantIdAndSchemaAndTableName(Mutation someRow) {
+        byte[][] rowKeyMetaData = new byte[3][];
+        getVarChars(someRow.getRow(), 3, rowKeyMetaData);
+        return ByteUtil.concat(rowKeyMetaData[0], rowKeyMetaData[1], rowKeyMetaData[2]);
+    }
+
+    public static byte[] getTenantIdAndSchemaAndTableName(Result result) {
+        byte[][] rowKeyMetaData = new byte[3][];
+        getVarChars(result.getRow(), 3, rowKeyMetaData);
+        return ByteUtil.concat(rowKeyMetaData[0], rowKeyMetaData[1], rowKeyMetaData[2]);
+    }
+
     public static void getTenantIdAndSchemaAndTableName(List<Mutation> tableMetadata, byte[][] rowKeyMetaData) {
         Mutation m = getTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);
     }
 
+    public static int getBaseColumnCount(List<Mutation> tableMetadata) {
+        int result = -1;
+        for (Mutation mutation : tableMetadata) {
+            for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
+                for (Cell cell : cells) {
+                    // compare using offsets
+                    if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES, 0,
+                        PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES.length) == 0)
+                    if (Bytes.contains(cell.getQualifierArray(), PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES)) {
+                        result = PInteger.INSTANCE.getCodec()
+                            .decodeInt(cell.getValueArray(), cell.getValueOffset(), SortOrder.ASC);
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    public static void mutatePutValue(Put somePut, byte[] family, byte[] qualifier, byte[] newValue) {
+        NavigableMap<byte[], List<Cell>> familyCellMap = somePut.getFamilyCellMap();
+        List<Cell> cells = familyCellMap.get(family);
+        List<Cell> newCells = Lists.newArrayList();
+        if (cells != null) {
+            for (Cell cell : cells) {
+                if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+                    qualifier, 0, qualifier.length) == 0) {
+                    Cell replacementCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+                        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), cell.getQualifierArray(),
+                        cell.getQualifierOffset(), cell.getQualifierLength(), cell.getTimestamp(),
+                        KeyValue.Type.codeToType(cell.getTypeByte()), newValue, 0, newValue.length);
+                    newCells.add(replacementCell);
+                } else {
+                    newCells.add(cell);
+                }
+            }
+            familyCellMap.put(family, newCells);
+        }
+    }
+
+    public static Put cloneDeleteToPutAndAddColumn(Delete delete, byte[] family, byte[] qualifier, byte[] value) {
+        NavigableMap<byte[], List<Cell>> familyCellMap = delete.getFamilyCellMap();
+        List<Cell> cells = familyCellMap.get(family);
+        Cell cell = Iterables.getFirst(cells, null);
+        if (cell == null) {
+            throw new RuntimeException("Empty cells for delete for family: " + Bytes.toStringBinary(family));
+        }
+        byte[] rowArray = new byte[cell.getRowLength()];
+        System.arraycopy(cell.getRowArray(), cell.getRowOffset(), rowArray, 0, cell.getRowLength());
+        Put put = new Put(rowArray, delete.getTimeStamp());
+        put.addColumn(family, qualifier, delete.getTimeStamp(), value);
+        return put;
+    }
+
+
     public static void getTenantIdAndFunctionName(List<Mutation> functionMetadata, byte[][] rowKeyMetaData) {
         Mutation m = getTableHeaderRow(functionMetadata);
         getVarChars(m.getRow(), 2, rowKeyMetaData);
     }
 
+    /**
+     * Only return the parent table name if it has the same tenant id and schema name as the current
+     * table (this is only used to lock the parent table of indexes)
+     */
     public static byte[] getParentTableName(List<Mutation> tableMetadata) {
         if (tableMetadata.size() == 1) {
             return null;
         }
         byte[][] rowKeyMetaData = new byte[3][];
+        // get the tenantId, schema name and table name for the current table
         getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
+        byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
         byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        // get the tenantId, schema name and table name for the parent table
         Mutation m = getParentTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);
-        if (   Bytes.compareTo(schemaName, rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]) == 0
-            && Bytes.compareTo(tableName, rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
+        if (Bytes.compareTo(tenantId, rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX]) == 0
+                && Bytes.compareTo(schemaName,
+                    rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]) == 0
+                && Bytes.compareTo(tableName,
+                    rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
             return null;
         }
         return rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
@@ -238,6 +322,24 @@ public class MetaDataUtil {
     }
 
     
+    public static ViewType getViewType(List<Mutation> tableMetaData, KeyValueBuilder builder,
+    	      ImmutableBytesWritable value) {
+    	        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
+    	            PhoenixDatabaseMetaData.VIEW_TYPE_BYTES, builder, value)) {
+    	            return ViewType.fromSerializedValue(value.get()[value.getOffset()]);
+    	        }
+    	        return null;
+    	    }
+    
+    public static int getSaltBuckets(List<Mutation> tableMetaData, KeyValueBuilder builder,
+      ImmutableBytesWritable value) {
+        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
+            PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES, builder, value)) {
+            return PInteger.INSTANCE.getCodec().decodeInt(value, SortOrder.getDefault());
+        }
+        return 0;
+    }
+    
     public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
         return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
     }
@@ -273,6 +375,35 @@ public class MetaDataUtil {
         return false;
     }
 
+    public static KeyValue getMutationValue(Mutation headerRow, byte[] key,
+        KeyValueBuilder builder) {
+        List<Cell> kvs = headerRow.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        if (kvs != null) {
+            for (Cell cell : kvs) {
+                KeyValue kv = org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cell);
+                if (builder.compareQualifier(kv, key, 0, key.length) ==0) {
+                    return kv;
+                }
+            }
+        }
+        return null;
+    }
+
+    public static boolean setMutationValue(Mutation headerRow, byte[] key,
+        KeyValueBuilder builder, KeyValue keyValue) {
+        List<Cell> kvs = headerRow.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        if (kvs != null) {
+            for (Cell cell : kvs) {
+                KeyValue kv = org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cell);
+                if (builder.compareQualifier(kv, key, 0, key.length) ==0) {
+                    KeyValueBuilder.addQuietly(headerRow, builder, keyValue);
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
     /**
      * Returns the first Put element in <code>tableMetaData</code>. There could be leading Delete elements before the
      * table header row
@@ -601,14 +732,17 @@ public class MetaDataUtil {
 	}
 
     public static LinkType getLinkType(Mutation tableMutation) {
-        List<Cell> kvs = tableMutation.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        return getLinkType(tableMutation.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES));
+    }
+
+    public static LinkType getLinkType(Collection<Cell> kvs) {
         if (kvs != null) {
             for (Cell kv : kvs) {
                 if (Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
-                        PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
-                        PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0) { return LinkType
-                                .fromSerializedValue(PUnsignedTinyint.INSTANCE.getCodec().decodeByte(kv.getValueArray(),
-                                        kv.getValueOffset(), SortOrder.getDefault())); }
+                    PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
+                    PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0) { return LinkType
+                    .fromSerializedValue(PUnsignedTinyint.INSTANCE.getCodec().decodeByte(kv.getValueArray(),
+                        kv.getValueOffset(), SortOrder.getDefault())); }
             }
         }
         return null;
@@ -674,7 +808,28 @@ public class MetaDataUtil {
         return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
     }
     
-    public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
+	public static List<Mutation> removeChildLinks(List<Mutation> catalogMutations) {
+		List<Mutation> childLinks = Lists.newArrayList();
+		Iterator<Mutation> iter = catalogMutations.iterator();
+		while (iter.hasNext()) {
+			Mutation m = iter.next();
+			for (KeyValue kv : m.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES)) {
+				// remove mutations of link type LinkType.CHILD_TABLE
+				if ((Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
+						PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
+						PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0)
+						&& ((Bytes.compareTo(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
+								LinkType.CHILD_TABLE.getSerializedValueAsByteArray(), 0,
+								LinkType.CHILD_TABLE.getSerializedValueAsByteArray().length) == 0))) {
+					childLinks.add(m);
+					iter.remove();
+				}
+			}
+		}
+		return childLinks;
+	}
+
+	public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
             ImmutableBytesWritable value) {
         if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData), PhoenixDatabaseMetaData.INDEX_TYPE_BYTES, builder,
                 value)) { return IndexType.fromSerializedValue(value.get()[value.getOffset()]); }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 6cf6e56..aee66aa 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
@@ -263,7 +263,6 @@ public class PhoenixRuntime {
                 String srcTable = execCmd.getSrcTable();
                 System.out.println("Starting upgrading table:" + srcTable + "... please don't kill it in between!!");
                 UpgradeUtil.upgradeTable(conn, srcTable);
-                UpgradeUtil.mapChildViewsToNamespace(conn, srcTable,props);
             } else if (execCmd.isUpgrade()) {
                 if (conn.getClientInfo(PhoenixRuntime.CURRENT_SCN_ATTRIB) != null) { throw new SQLException(
                         "May not specify the CURRENT_SCN property when upgrading"); }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 dd00a69..48d2a5c 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
@@ -22,13 +22,17 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Strings.isNullOrEmpty;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE;
+import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
@@ -274,7 +278,7 @@ public class SchemaUtil {
      * @param tableName
      */
     public static byte[] getTableKey(byte[] tenantId, byte[] schemaName, byte[] tableName) {
-        return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName, QueryConstants.SEPARATOR_BYTE_ARRAY, tableName);
+        return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId, SEPARATOR_BYTE_ARRAY, schemaName, SEPARATOR_BYTE_ARRAY, tableName);
     }
 
     /**
@@ -283,32 +287,32 @@ public class SchemaUtil {
      * @param functionName
      */
     public static byte[] getFunctionKey(byte[] tenantId, byte[] functionName) {
-        return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, functionName);
+        return ByteUtil.concat(tenantId, SEPARATOR_BYTE_ARRAY, functionName);
     }
 
     public static byte[] getKeyForSchema(String tenantId, String schemaName) {
         return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
-                QueryConstants.SEPARATOR_BYTE_ARRAY,
+                SEPARATOR_BYTE_ARRAY,
                 schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName));
     }
 
     public static byte[] getTableKey(String tenantId, String schemaName, String tableName) {
-        return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName));
+        return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName));
     }
 
     public static byte[] getColumnKey(String tenantId, String schemaName, String tableName, String columnName, String familyName) {
         Preconditions.checkNotNull(columnName,"Column name cannot be null");
         if (familyName == null) {
             return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
-                    QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), 
-                    QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
-                    QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName));
+                    SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName),
+                    SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
+                    SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName));
         }
         return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
-                QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), 
-                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
-                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName),
-                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(familyName));
+                SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName),
+                SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
+                SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName),
+                SEPARATOR_BYTE_ARRAY, Bytes.toBytes(familyName));
     }
 
     public static String getTableName(String schemaName, String tableName) {
@@ -536,6 +540,11 @@ public class SchemaUtil {
                 || Bytes.compareTo(tableName, SchemaUtil
                         .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
     }
+    
+    public static boolean isChildLinkTable(byte[] tableName) {
+        return Bytes.compareTo(tableName, SYSTEM_CHILD_LINK_NAME_BYTES) == 0 || Bytes.compareTo(tableName,
+                SchemaUtil.getPhysicalTableName(SYSTEM_CHILD_LINK_NAME_BYTES, true).getName()) == 0;
+    }
 
     public static boolean isSequenceTable(PTable table) {
         return PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME.equals(table.getName().getString());
@@ -603,7 +612,7 @@ public class SchemaUtil {
                 if (pos == pkColumns.size() - 1) {
                     break;
                 }
-                while (offset < maxOffset && split[offset] != QueryConstants.SEPARATOR_BYTE) {
+                while (offset < maxOffset && split[offset] != SEPARATOR_BYTE) {
                     offset++;
                 }
                 if (offset == maxOffset) {
@@ -645,7 +654,7 @@ public class SchemaUtil {
         PhoenixConnection metaConnection = null;
         Statement stmt = null;
         try {
-            metaConnection = new PhoenixConnection(conn.getQueryServices(), conn, scn);
+            metaConnection = new PhoenixConnection(conn, scn);
             try {
                 stmt = metaConnection.createStatement();
                 stmt.executeUpdate("ALTER TABLE SYSTEM.\"TABLE\" ADD IF NOT EXISTS " + columnDef);
@@ -893,7 +902,7 @@ public class SchemaUtil {
      * @return the byte to use as the separator
      */
     public static byte getSeparatorByte(boolean rowKeyOrderOptimizable, boolean isNullValue, SortOrder sortOrder) {
-        return !rowKeyOrderOptimizable || isNullValue || sortOrder == SortOrder.ASC ? QueryConstants.SEPARATOR_BYTE : QueryConstants.DESC_SEPARATOR_BYTE;
+        return !rowKeyOrderOptimizable || isNullValue || sortOrder == SortOrder.ASC ? SEPARATOR_BYTE : QueryConstants.DESC_SEPARATOR_BYTE;
     }
     
     public static byte getSeparatorByte(boolean rowKeyOrderOptimizable, boolean isNullValue, Field f) {
@@ -1157,7 +1166,11 @@ public class SchemaUtil {
         }
     }
 
-    public static boolean hasGlobalIndex(PTable table) {
+	public static int getIsNullableInt(boolean isNullable) {
+		return isNullable ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls;
+	}
+
+	public static boolean hasGlobalIndex(PTable table) {
         for (PTable index : table.getIndexes()) {
             if (index.getIndexType() == IndexType.GLOBAL) {
                 return true;


[49/50] [abbrv] phoenix git commit: PHOENIX-4892 Unable to start load balancer with queryserver

Posted by ja...@apache.org.
PHOENIX-4892 Unable to start load balancer with queryserver

Added phoenix_loadbalancer_jar path to phoenix_utils.py,
added phoenix_loadbalancer_jar to classpath in queryserver.py,
added service for registry in meta-inf

Closes #343

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: e5379cd1dc6b0ca40ed8a0524332acf75430b15f
Parents: 7fa11ff
Author: Vitaliy <vi...@mail.ru>
Authored: Tue Sep 18 15:05:55 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Sep 18 17:28:42 2018 -0400

----------------------------------------------------------------------
 bin/phoenix_utils.py                                          | 7 +++++++
 bin/queryserver.py                                            | 4 ++--
 phoenix-load-balancer/pom.xml                                 | 1 +
 .../services/org.apache.phoenix.queryserver.register.Registry | 1 +
 4 files changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e5379cd1/bin/phoenix_utils.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_utils.py b/bin/phoenix_utils.py
index b521277..aa04a5b 100755
--- a/bin/phoenix_utils.py
+++ b/bin/phoenix_utils.py
@@ -71,6 +71,7 @@ def setPath():
     PHOENIX_CLIENT_JAR_PATTERN = "phoenix-*-client.jar"
     PHOENIX_THIN_CLIENT_JAR_PATTERN = "phoenix-*-thin-client.jar"
     PHOENIX_QUERYSERVER_JAR_PATTERN = "phoenix-*-queryserver.jar"
+    PHOENIX_LOADBALANCER_JAR_PATTERN = "phoenix-load-balancer-*[!t][!e][!s][!t][!s].jar"
     PHOENIX_TRACESERVER_JAR_PATTERN = "phoenix-tracing-webapp-*-runnable.jar"
     PHOENIX_TESTS_JAR_PATTERN = "phoenix-core-*-tests*.jar"
     PHOENIX_PHERF_JAR_PATTERN = "phoenix-pherf-*-minimal*.jar"
@@ -162,6 +163,11 @@ def setPath():
     if phoenix_queryserver_jar == "":
         phoenix_queryserver_jar = findFileInPathWithoutRecursion(PHOENIX_QUERYSERVER_JAR_PATTERN, os.path.join(current_dir, ".."))
 
+    global phoenix_loadbalancer_jar
+    phoenix_loadbalancer_jar = find(PHOENIX_LOADBALANCER_JAR_PATTERN, os.path.join(current_dir, "..", "phoenix-loadbalancer", "target", "*"))
+    if phoenix_loadbalancer_jar == "":
+        phoenix_loadbalancer_jar = findFileInPathWithoutRecursion(PHOENIX_LOADBALANCER_JAR_PATTERN, os.path.join(current_dir, ".."))
+
     global phoenix_traceserver_jar
     phoenix_traceserver_jar = find(PHOENIX_TRACESERVER_JAR_PATTERN, os.path.join(current_dir, "..", "phoenix-tracing-webapp", "target", "*"))
     if phoenix_traceserver_jar == "":
@@ -218,5 +224,6 @@ if __name__ == "__main__":
     print "hadoop_hdfs_jar:", hadoop_hdfs_jar
     print "testjar:", testjar
     print "phoenix_queryserver_jar:", phoenix_queryserver_jar
+    print "phoenix_loadbalancer_jar:", phoenix_loadbalancer_jar
     print "phoenix_thin_client_jar:", phoenix_thin_client_jar
     print "hadoop_classpath:", hadoop_classpath 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e5379cd1/bin/queryserver.py
----------------------------------------------------------------------
diff --git a/bin/queryserver.py b/bin/queryserver.py
index 711bcc4..0c07b3b 100755
--- a/bin/queryserver.py
+++ b/bin/queryserver.py
@@ -120,8 +120,8 @@ out_file_path = os.path.join(log_dir, phoenix_out_file)
 
 # The command is run through subprocess so environment variables are automatically inherited
 java_cmd = '%(java)s -cp ' + hbase_config_path + os.pathsep + hadoop_config_path + os.pathsep + \
-    phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.phoenix_queryserver_jar + \
-    os.pathsep + hadoop_classpath + \
+    phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.phoenix_loadbalancer_jar + \
+    os.pathsep + phoenix_utils.phoenix_queryserver_jar + os.pathsep + hadoop_classpath + \
     " -Dproc_phoenixserver" + \
     " -Dlog4j.configuration=file:" + os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " -Dpsql.root.logger=%(root_logger)s" + \

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e5379cd1/phoenix-load-balancer/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-load-balancer/pom.xml b/phoenix-load-balancer/pom.xml
index 7cffb58..48f7b03 100644
--- a/phoenix-load-balancer/pom.xml
+++ b/phoenix-load-balancer/pom.xml
@@ -75,6 +75,7 @@
         <configuration>
           <excludes>
             <exclude>src/main/resources/META-INF/services/org.apache.phoenix.loadbalancer.service.LoadBalanceZookeeperConf</exclude>
+            <exclude>src/main/resources/META-INF/services/org.apache.phoenix.queryserver.register.Registry</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e5379cd1/phoenix-load-balancer/src/main/resources/META-INF/services/org.apache.phoenix.queryserver.register.Registry
----------------------------------------------------------------------
diff --git a/phoenix-load-balancer/src/main/resources/META-INF/services/org.apache.phoenix.queryserver.register.Registry b/phoenix-load-balancer/src/main/resources/META-INF/services/org.apache.phoenix.queryserver.register.Registry
new file mode 100644
index 0000000..05e1006
--- /dev/null
+++ b/phoenix-load-balancer/src/main/resources/META-INF/services/org.apache.phoenix.queryserver.register.Registry
@@ -0,0 +1 @@
+org.apache.phoenix.queryserver.register.ZookeeperRegistry
\ No newline at end of file


[35/50] [abbrv] phoenix git commit: PHOENIX-4798 Update encoded col qualifiers on the base table correctly

Posted by ja...@apache.org.
PHOENIX-4798 Update encoded col qualifiers on the base table correctly


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

Branch: refs/heads/omid2
Commit: b0cc455c93df4bd13a7db5a8a21496f2eb170500
Parents: 46f50d2
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Aug 15 12:23:56 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Mon Aug 27 15:30:26 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/ViewIT.java | 184 +++++++++++++------
 .../coprocessor/MetaDataEndpointImpl.java       |  74 +++++---
 .../PhoenixMetaDataCoprocessorHost.java         |   2 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  33 +++-
 4 files changed, 208 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b0cc455c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index fda9490..c1a7ff5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -45,6 +45,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -59,19 +60,16 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.coprocessor.BaseMetaDataEndpointObserver;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
 import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
@@ -98,36 +96,39 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class ViewIT extends SplitSystemCatalogIT {
 
     protected String tableDDLOptions;
     protected boolean transactional;
+    protected boolean columnEncoded;
     
-    private static final String FAILED_VIEWNAME = "FAILED_VIEW";
-    private static final byte[] FAILED_ROWKEY_BYTES =
-            SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2), Bytes.toBytes(FAILED_VIEWNAME));
-    private static final String SLOW_VIEWNAME_PREFIX = "SLOW_VIEW";
-    private static final byte[] SLOW_ROWKEY_PREFIX_BYTES =
-            SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2),
-                Bytes.toBytes(SLOW_VIEWNAME_PREFIX));
+    private static final String FAILED_VIEWNAME = SchemaUtil.getTableName(SCHEMA2, "FAILED_VIEW");
+    private static final String SLOW_VIEWNAME_PREFIX = SchemaUtil.getTableName(SCHEMA2, "SLOW_VIEW");
 
     private static volatile CountDownLatch latch1 = null;
     private static volatile CountDownLatch latch2 = null;
 
-    public ViewIT(boolean transactional) {
+    public ViewIT(boolean transactional, boolean columnEncoded) {
         StringBuilder optionBuilder = new StringBuilder();
         this.transactional = transactional;
+        this.columnEncoded = columnEncoded;
         if (transactional) {
             optionBuilder.append(" TRANSACTIONAL=true ");
         }
+        if (!columnEncoded) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
         this.tableDDLOptions = optionBuilder.toString();
     }
 
-    @Parameters(name = "transactional = {0}")
-    public static Collection<Boolean> data() {
-        return Arrays.asList(new Boolean[] { false, true });
+    @Parameters(name="ViewIT_transactional={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] { 
+            { true, false }, { true, true },
+            { false, false }, { false, true }});
     }
     
     @BeforeClass
@@ -136,7 +137,9 @@ public class ViewIT extends SplitSystemCatalogIT {
         Map<String, String> props = Collections.emptyMap();
         boolean splitSystemCatalog = (driver == null);
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
-        serverProps.put("hbase.coprocessor.region.classes", TestMetaDataRegionObserver.class.getName());
+        serverProps.put(QueryServices.PHOENIX_ACLS_ENABLED, "true");
+        serverProps.put(PhoenixMetaDataCoprocessorHost.PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
+            TestMetaDataRegionObserver.class.getName());
         serverProps.put("hbase.coprocessor.abortonerror", "false");
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(props.entrySet().iterator()));
         // Split SYSTEM.CATALOG once after the mini-cluster is started
@@ -145,17 +148,36 @@ public class ViewIT extends SplitSystemCatalogIT {
         }
     }
     
-    public static class TestMetaDataRegionObserver extends BaseRegionObserver {
+    public static class TestMetaDataRegionObserver extends BaseMetaDataEndpointObserver {
+        
+        @Override
+        public void preAlterTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+                String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType type) throws IOException{
+            processTable(tableName);
+        }
+        
+        @Override
+        public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+                String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+                Set<byte[]> familySet, Set<TableName> indexes) throws IOException {
+            processTable(tableName);
+        }
+
         @Override
-        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
-                MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
-            if (shouldFail(c, miniBatchOp.getOperation(0))) {
+        public void preDropTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+                String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+                List<PTable> indexes) throws IOException {
+            processTable(tableName);
+        }
+
+        private void processTable(String tableName) throws DoNotRetryIOException {
+            if (tableName.equals(FAILED_VIEWNAME)) {
                 // throwing anything other than instances of IOException result
                 // in this coprocessor being unloaded
                 // DoNotRetryIOException tells HBase not to retry this mutation
                 // multiple times
                 throw new DoNotRetryIOException();
-            } else if (shouldSlowDown(c, miniBatchOp.getOperation(0))) {
+            } else if (tableName.startsWith(SLOW_VIEWNAME_PREFIX)) {
                 // simulate a slow write to SYSTEM.CATALOG
                 if (latch1 != null) {
                     latch1.countDown();
@@ -172,20 +194,7 @@ public class ViewIT extends SplitSystemCatalogIT {
                 }
             }
         }
-
-        private boolean shouldFail(ObserverContext<RegionCoprocessorEnvironment> c, Mutation m) {
-            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
-            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
-                    && (Bytes.equals(FAILED_ROWKEY_BYTES, m.getRow()));
-        }
-
-        private boolean shouldSlowDown(ObserverContext<RegionCoprocessorEnvironment> c,
-                Mutation m) {
-            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
-            byte[] rowKeyPrefix = Arrays.copyOf(m.getRow(), SLOW_ROWKEY_PREFIX_BYTES.length);
-            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
-                    && (Bytes.equals(SLOW_ROWKEY_PREFIX_BYTES, rowKeyPrefix));
-        }
+        
     }
     
     @Test
@@ -598,9 +607,6 @@ public class ViewIT extends SplitSystemCatalogIT {
     public void testViewAndTableAndDropCascadeWithIndexes() throws Exception {
         // Setup - Tables and Views with Indexes
         Connection conn = DriverManager.getConnection(getUrl());
-        if (tableDDLOptions.length()!=0)
-            tableDDLOptions+=",";
-        tableDDLOptions+="IMMUTABLE_ROWS=true";
         String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
         String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
@@ -1311,7 +1317,7 @@ public class ViewIT extends SplitSystemCatalogIT {
     public void testChildViewCreationFails() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, FAILED_VIEWNAME);
+        String fullViewName1 = FAILED_VIEWNAME;
         String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
         
         String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
@@ -1343,9 +1349,7 @@ public class ViewIT extends SplitSystemCatalogIT {
     public void testConcurrentViewCreationAndTableDrop() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 =
-                    SchemaUtil.getTableName(SCHEMA2,
-                        SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
             String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
             latch1 = new CountDownLatch(1);
             latch2 = new CountDownLatch(1);
@@ -1392,12 +1396,12 @@ public class ViewIT extends SplitSystemCatalogIT {
     }
 
     @Test
-    public void testConcurrentAddColumn() throws Exception {
+    public void testConcurrentAddSameColumnDifferentType() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
+            latch1 = null;
+            latch2 = null;
             String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 =
-                    SchemaUtil.getTableName(SCHEMA2,
-                        SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
             String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
             // create base table
             String tableDdl =
@@ -1422,10 +1426,10 @@ public class ViewIT extends SplitSystemCatalogIT {
                 }
             });
 
-            // add a column to the view in a separate thread (which will take
-            // some time to complete)
+            // add a column with the same name and different type to the view in a separate thread
+            // (which will take some time to complete)
             Future<Exception> future = executorService.submit(new AddColumnRunnable(fullViewName1));
-            // wait till the thread makes the rpc to create the view
+            // wait till the thread makes the rpc to add the column
             boolean result = latch1.await(2, TimeUnit.MINUTES);
             if (!result) {
                 fail("The create view rpc look too long");
@@ -1451,6 +1455,82 @@ public class ViewIT extends SplitSystemCatalogIT {
             conn.createStatement().execute(tableDdl);
         }
     }
+    
+    @Test
+    public void testConcurrentAddDifferentColumn() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            latch1 = null;
+            latch2 = null;
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
+            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            String fullViewName3 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
+            // create base table
+            String tableDdl =
+                    "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                            + tableDDLOptions;
+            conn.createStatement().execute(tableDdl);
+            // create a two views
+            String ddl =
+                    "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
+                            + fullTableName + " WHERE k = 6";
+            conn.createStatement().execute(ddl);
+            ddl =
+                    "CREATE VIEW " + fullViewName3 + " (v2 VARCHAR) AS SELECT * FROM "
+                            + fullTableName + " WHERE k = 7";
+            conn.createStatement().execute(ddl);
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // add a column to a view in a separate thread (we slow this operation down)
+            Future<Exception> future = executorService.submit(new AddColumnRunnable(fullViewName1));
+            // wait till the thread makes the rpc to add the column
+            boolean result = latch1.await(2, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The alter view rpc look too long");
+            }
+            tableDdl = "ALTER VIEW " + fullViewName3 + " ADD v4 INTEGER";
+            try {
+                // add a column to another view 
+                conn.createStatement().execute(tableDdl);
+                if (columnEncoded) {
+                    // this should fail as the previous add column is still not complete
+                    fail(
+                        "Adding columns to two different views concurrently where the base table uses encoded column should fail");
+                }
+            } catch (ConcurrentTableMutationException e) {
+                if (!columnEncoded) {
+                    // this should not fail as we don't need to update the parent table for non
+                    // column encoded tables
+                    fail(
+                        "Adding columns to two different views concurrently where the base table does not use encoded columns should succeed");
+                }
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            // if the base table uses column encoding then the add column operation for fullViewName1 fails
+            assertNull(e);
+
+            // add a the same column to the another view  to ensure that the cell used
+            // to prevent concurrent modifications was removed
+            ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " 
+                    + fullTableName + " WHERE k = 6";
+            conn.createStatement().execute(ddl);
+            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
+            conn.createStatement().execute(tableDdl);
+        }
+    }
 
     private class CreateViewRunnable implements Callable<Exception> {
         private final String fullTableName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b0cc455c/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 5e8a5dc..e748115 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
@@ -2208,8 +2208,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 MetaDataResponse response =
                         processRemoteRegionMutations(
                             PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                            childLinkMutations, fullTableName,
-                            MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
+                            childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
                 if (response != null) {
                     done.run(response);
                     return;
@@ -2229,8 +2228,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         response =
                                 processRemoteRegionMutations(
                                     PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
-                                    remoteMutations, fullTableName,
-                                    MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                                    remoteMutations, MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                        clearParentTableFromCache(clientTimeStamp,
+                            parentTable.getSchemaName() != null
+                                    ? parentTable.getSchemaName().getBytes()
+                                    : ByteUtil.EMPTY_BYTE_ARRAY,
+                            parentTable.getName().getBytes());
                         if (response != null) {
                             done.run(response);
                             return;
@@ -2484,8 +2487,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 MetaDataResponse response =
                         processRemoteRegionMutations(
                             PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                            childLinkMutations, SchemaUtil.getTableName(schemaName, tableName),
-                            MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
+                            childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
                 if (response!=null) {
                     done.run(response);
                     return;
@@ -2521,8 +2523,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private MetaDataResponse processRemoteRegionMutations(byte[] systemTableName,
-            List<Mutation> remoteMutations, String tableName,
-            MetaDataProtos.MutationCode mutationCode) throws IOException {
+            List<Mutation> remoteMutations, MetaDataProtos.MutationCode mutationCode) throws IOException {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         try (Table hTable =
                 env.getTable(
@@ -2780,7 +2781,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             .getEncodingScheme() != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) {
                         processRemoteRegionMutations(
                             PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, remoteMutations,
-                            fullTableName, MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                            MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                        clearParentTableFromCache(clientTimeStamp,
+                            table.getParentSchemaName() != null
+                                    ? table.getParentSchemaName().getBytes()
+                                    : ByteUtil.EMPTY_BYTE_ARRAY,
+                            table.getParentTableName().getBytes());
                     }
                     else {
                         String msg = "Found unexpected mutations while adding or dropping column to "+fullTableName;
@@ -2815,6 +2821,25 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             return null; // impossible
         }
     }
+
+    /**
+     * Removes the table from the server side cache
+     */
+    private void clearParentTableFromCache(long clientTimeStamp, byte[] schemaName, byte[] tableName) throws SQLException {
+        // remove the parent table from the metadata cache as we just mutated the table
+        Properties props = new Properties();
+        if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
+            props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
+        }
+        try (PhoenixConnection connection =
+                QueryUtil.getConnectionOnServer(props, env.getConfiguration())
+                        .unwrap(PhoenixConnection.class)) {
+            ConnectionQueryServices queryServices = connection.getQueryServices();
+            queryServices.clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY, schemaName, tableName,
+                clientTimeStamp);
+        } catch (ClassNotFoundException e) {
+        }
+    }
     
     private static boolean isDivergedView(PTable view) {
         return view.getBaseColumnCount() == QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
@@ -3205,28 +3230,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             } 
                         }
                     } 
-                    if (type == PTableType.VIEW
-                            && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
-                        /*
-                         * When adding a column to a view that uses encoded column name scheme, we
-                         * need to modify the CQ counters stored in the view's physical table. So to
-                         * make sure clients get the latest PTable, we need to invalidate the cache
-                         * entry.
-                         */
-                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
-                                .getPhysicalTableRowForView(table)));
-
-
-
-                    }
+                    boolean addingCol = false;
                     for (Mutation m : tableMetaData) {
                         byte[] key = m.getRow();
                         boolean addingPKColumn = false;
                         int pkCount = getVarChars(key, rowKeyMetaData);
+                        // this means we have are adding a column 
                         if (pkCount > COLUMN_NAME_INDEX
                                 && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
                                 && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
                             try {
+                                addingCol = true;
                                 if (pkCount > FAMILY_NAME_INDEX
                                         && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
                                     PColumnFamily family =
@@ -3291,6 +3305,20 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         }
                     }
                     tableMetaData.addAll(additionalTableMetadataMutations);
+                    if (type == PTableType.VIEW
+                                && EncodedColumnsUtil.usesEncodedColumnNames(table) && addingCol
+                                && !table.isAppendOnlySchema()) {
+                                // When adding a column to a view that uses encoded column name
+                                // scheme, we need to modify the CQ counters stored in the view's
+                                // physical table. So to make sure clients get the latest PTable, we
+                                // need to invalidate the cache entry.
+                                // If the table uses APPEND_ONLY_SCHEMA we use the position of the
+                                // column as the encoded column qualifier and so we don't need to
+                                // update the CQ counter in the view physical table (see
+                                // PHOENIX-4737)
+                                invalidateList.add(new ImmutableBytesPtr(
+                                        MetaDataUtil.getPhysicalTableRowForView(table)));
+                    }
                     return null;
                 }
             }, request.getClientVersion());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b0cc455c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
index 15b0020..059bca1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
@@ -104,7 +104,7 @@ public class PhoenixMetaDataCoprocessorHost
     /**
      * Encapsulation of the environment of each coprocessor
      */
-    static class PhoenixMetaDataControllerEnvironment extends CoprocessorHost.Environment
+    public static class PhoenixMetaDataControllerEnvironment extends CoprocessorHost.Environment
             implements RegionCoprocessorEnvironment {
 
         private RegionCoprocessorEnvironment env;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b0cc455c/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 c714eab..1114463 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
@@ -3321,6 +3321,7 @@ public class MetaDataClient {
         String physicalTableName =
                 SchemaUtil.getTableNameFromFullName(physicalName.getString());
         Set<String> acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3);
+        boolean acquiredMutex = false;
         try {
             connection.setAutoCommit(false);
 
@@ -3599,17 +3600,26 @@ public class MetaDataClient {
                     }
                 }
 
-                boolean acquiredMutex = true;
-                for (PColumn pColumn : columns) {
-                    // acquire the mutex using the global physical table name to
-                    // prevent creating the same column on a table or view with
-                    // a conflicting type etc
-                    acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName,
-                        pColumn.getName().getString());
+                if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
+                    // for tables that use column encoding acquire a mutex on the base table as we
+                    // need to update the encoded column qualifier counter on the base table
+                    acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName, null);
                     if (!acquiredMutex) {
                         throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
                     }
-                    acquiredColumnMutexSet.add(pColumn.getName().getString());
+                }
+                else {
+                    for (PColumn pColumn : columns) {
+                        // acquire the mutex using the global physical table name to
+                        // prevent creating the same column on a table or view with
+                        // a conflicting type etc
+                        acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName,
+                            pColumn.getName().getString());
+                        if (!acquiredMutex) {
+                            throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
+                        }
+                        acquiredColumnMutexSet.add(pColumn.getName().getString());
+                    }
                 }
                 MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded, columns);
                 try {
@@ -3681,7 +3691,12 @@ public class MetaDataClient {
             }
         } finally {
             connection.setAutoCommit(wasAutoCommit);
-            if (!acquiredColumnMutexSet.isEmpty()) {
+            if (EncodedColumnsUtil.usesEncodedColumnNames(table) && acquiredMutex) {
+                // release the mutex on the physical table (used to prevent concurrent conflicting
+                // add column changes)
+                deleteCell(null, physicalSchemaName, physicalTableName, null);
+            }
+            else if (!acquiredColumnMutexSet.isEmpty()) {
                 for (String columnName : acquiredColumnMutexSet) {
                     // release the mutex (used to prevent concurrent conflicting add column changes)
                     deleteCell(null, physicalSchemaName, physicalTableName, columnName);


[18/50] [abbrv] phoenix git commit: PHOENIX-4822 Ensure the provided timezone is used client-side (Jaanai Zhang)

Posted by ja...@apache.org.
PHOENIX-4822 Ensure the provided timezone is used client-side (Jaanai Zhang)


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

Branch: refs/heads/omid2
Commit: 88bf64d054967ca6a1f8331a480354ae249f2f4f
Parents: 6f5926b
Author: Josh Elser <el...@apache.org>
Authored: Tue Jul 31 15:53:11 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jul 31 16:17:33 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DateTimeIT.java  | 77 ++++++++++++++++++++
 .../phoenix/compile/StatementContext.java       | 11 +--
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  8 +-
 .../java/org/apache/phoenix/util/DateUtil.java  | 22 +++---
 4 files changed, 101 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/88bf64d0/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index c976114..cc7c7a7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -54,12 +54,19 @@ import java.text.Format;
 import java.util.Calendar;
 import java.util.GregorianCalendar;
 import java.util.Properties;
+import java.util.TimeZone;
 
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PTime;
 import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.DateUtil;
@@ -1880,4 +1887,74 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
+
+    @Test
+    public void testDateFormatTimeZone()throws Exception {
+        String[] timeZoneIDs = {DateUtil.DEFAULT_TIME_ZONE_ID, "Asia/Yerevan", "Australia/Adelaide", "Asia/Tokyo"};
+        for (String timeZoneID : timeZoneIDs) {
+            testDateFormatTimeZone(timeZoneID);
+        }
+    }
+
+    public void testDateFormatTimeZone(String timeZoneId) throws Exception {
+        Properties props = new Properties();
+        props.setProperty("phoenix.query.dateFormatTimeZone", timeZoneId);
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+
+        String tableName = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName +
+                " (k1 INTEGER PRIMARY KEY," +
+                " v_date DATE," +
+                " v_time TIME," +
+                " v_timestamp TIMESTAMP)";
+        try {
+            conn1.createStatement().execute(ddl);
+
+            PhoenixConnection pConn = conn1.unwrap(PhoenixConnection.class);
+            verifyTimeZoneIDWithConn(pConn, PDate.INSTANCE, timeZoneId);
+            verifyTimeZoneIDWithConn(pConn, PTime.INSTANCE, timeZoneId);
+            verifyTimeZoneIDWithConn(pConn, PTimestamp.INSTANCE, timeZoneId);
+
+            Calendar cal = Calendar.getInstance(TimeZone.getTimeZone(timeZoneId));
+            cal.setTime(date);
+            String dateStr = DateUtil.getDateFormatter(DateUtil.DEFAULT_MS_DATE_FORMAT).format(date);
+
+            String dml = "UPSERT INTO " + tableName + " VALUES (" +
+                    "1," +
+                    "'" + dateStr + "'," +
+                    "'" + dateStr + "'," +
+                    "'" + dateStr + "'" +
+                    ")";
+            conn1.createStatement().execute(dml);
+            conn1.commit();
+
+            PhoenixStatement stmt = conn1.createStatement().unwrap(PhoenixStatement.class);
+            ResultSet rs = stmt.executeQuery("SELECT v_date, v_time, v_timestamp FROM " + tableName);
+
+            assertTrue(rs.next());
+            assertEquals(rs.getDate(1).toString(), new Date(cal.getTimeInMillis()).toString());
+            assertEquals(rs.getTime(2).toString(), new Time(cal.getTimeInMillis()).toString());
+            assertEquals(rs.getTimestamp(3).getTime(), cal.getTimeInMillis());
+            assertFalse(rs.next());
+
+            StatementContext stmtContext = stmt.getQueryPlan().getContext();
+            verifyTimeZoneIDWithFormatter(stmtContext.getDateFormatter(), timeZoneId);
+            verifyTimeZoneIDWithFormatter(stmtContext.getTimeFormatter(), timeZoneId);
+            verifyTimeZoneIDWithFormatter(stmtContext.getTimestampFormatter(), timeZoneId);
+
+            stmt.close();
+        } finally {
+            conn1.close();
+        }
+    }
+
+    private void verifyTimeZoneIDWithConn(PhoenixConnection conn, PDataType dataType, String timeZoneId) {
+        Format formatter = conn.getFormatter(dataType);
+        verifyTimeZoneIDWithFormatter(formatter, timeZoneId);
+    }
+
+    private void verifyTimeZoneIDWithFormatter(Format formatter, String timeZoneId) {
+        assertTrue(formatter instanceof FastDateFormat);
+        assertEquals(((FastDateFormat)formatter).getTimeZone().getID(), timeZoneId);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/88bf64d0/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 fe60bb9..eb195c2 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
@@ -120,14 +120,15 @@ public class StatementContext {
         this.expressions = new ExpressionManager();
         PhoenixConnection connection = statement.getConnection();
         ReadOnlyProps props = connection.getQueryServices().getProps();
+        String timeZoneID = props.get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB,
+                DateUtil.DEFAULT_TIME_ZONE_ID);
         this.dateFormat = props.get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
-        this.dateFormatter = DateUtil.getDateFormatter(dateFormat);
+        this.dateFormatter = DateUtil.getDateFormatter(dateFormat, timeZoneID);
         this.timeFormat = props.get(QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
-        this.timeFormatter = DateUtil.getTimeFormatter(timeFormat);
+        this.timeFormatter = DateUtil.getTimeFormatter(timeFormat, timeZoneID);
         this.timestampFormat = props.get(QueryServices.TIMESTAMP_FORMAT_ATTRIB, DateUtil.DEFAULT_TIMESTAMP_FORMAT);
-        this.timestampFormatter = DateUtil.getTimestampFormatter(timestampFormat);
-        this.dateFormatTimeZone = DateUtil.getTimeZone(props.get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB,
-                DateUtil.DEFAULT_TIME_ZONE_ID));
+        this.timestampFormatter = DateUtil.getTimestampFormatter(timestampFormat, timeZoneID);
+        this.dateFormatTimeZone = DateUtil.getTimeZone(timeZoneID);
         this.numberFormat = props.get(QueryServices.NUMBER_FORMAT_ATTRIB, NumberUtil.DEFAULT_NUMBER_FORMAT);
         this.tempPtr = new ImmutableBytesWritable();
         this.currentTable = resolver != null && !resolver.getTables().isEmpty() ? resolver.getTables().get(0) : null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/88bf64d0/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 3d9b261..6da579f 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
@@ -327,9 +327,11 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
         int maxSizeBytes = this.services.getProps().getInt(
                 QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,
                 QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
-        Format dateFormat = DateUtil.getDateFormatter(datePattern);
-        Format timeFormat = DateUtil.getDateFormatter(timePattern);
-        Format timestampFormat = DateUtil.getDateFormatter(timestampPattern);
+        String timeZoneID = this.services.getProps().get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB,
+                DateUtil.DEFAULT_TIME_ZONE_ID);
+        Format dateFormat = DateUtil.getDateFormatter(datePattern, timeZoneID);
+        Format timeFormat = DateUtil.getDateFormatter(timePattern, timeZoneID);
+        Format timestampFormat = DateUtil.getDateFormatter(timestampPattern, timeZoneID);
         formatters.put(PDate.INSTANCE, dateFormat);
         formatters.put(PTime.INSTANCE, timeFormat);
         formatters.put(PTimestamp.INSTANCE, timestampFormat);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/88bf64d0/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
index 9e37eca..f67f152 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
@@ -97,7 +97,7 @@ public class DateUtil {
     
     public static TimeZone getTimeZone(String timeZoneId) {
         TimeZone parserTimeZone;
-        if (timeZoneId == null) {
+        if (timeZoneId == null || timeZoneId.equals(DateUtil.DEFAULT_TIME_ZONE_ID)) {
             parserTimeZone = DateUtil.DEFAULT_TIME_ZONE;
         } else if (LOCAL_TIME_ZONE_ID.equalsIgnoreCase(timeZoneId)) {
             parserTimeZone = TimeZone.getDefault();
@@ -164,21 +164,25 @@ public class DateUtil {
     }
 
     public static Format getDateFormatter(String pattern) {
-        return DateUtil.DEFAULT_DATE_FORMAT.equals(pattern)
+        return getDateFormatter(pattern, DateUtil.DEFAULT_TIME_ZONE_ID);
+    }
+
+    public static Format getDateFormatter(String pattern, String timeZoneID) {
+        return DateUtil.DEFAULT_DATE_FORMAT.equals(pattern) && DateUtil.DEFAULT_TIME_ZONE_ID.equals(timeZoneID)
                 ? DateUtil.DEFAULT_DATE_FORMATTER
-                : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE);
+                : FastDateFormat.getInstance(pattern, getTimeZone(timeZoneID));
     }
 
-    public static Format getTimeFormatter(String pattern) {
-        return DateUtil.DEFAULT_TIME_FORMAT.equals(pattern)
+    public static Format getTimeFormatter(String pattern, String timeZoneID) {
+        return DateUtil.DEFAULT_TIME_FORMAT.equals(pattern) && DateUtil.DEFAULT_TIME_ZONE_ID.equals(timeZoneID)
                 ? DateUtil.DEFAULT_TIME_FORMATTER
-                : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE);
+                : FastDateFormat.getInstance(pattern, getTimeZone(timeZoneID));
     }
 
-    public static Format getTimestampFormatter(String pattern) {
-        return DateUtil.DEFAULT_TIMESTAMP_FORMAT.equals(pattern)
+    public static Format getTimestampFormatter(String pattern, String timeZoneID) {
+        return DateUtil.DEFAULT_TIMESTAMP_FORMAT.equals(pattern) && DateUtil.DEFAULT_TIME_ZONE_ID.equals(timeZoneID)
                 ? DateUtil.DEFAULT_TIMESTAMP_FORMATTER
-                : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE);
+                : FastDateFormat.getInstance(pattern, getTimeZone(timeZoneID));
     }
 
     private static long parseDateTime(String dateTimeValue) {


[07/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index e968e99..4433e12 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -35,8 +35,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 
-import jline.internal.Log;
-
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
@@ -75,25 +73,27 @@ import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.primitives.Doubles;
 
+import jline.internal.Log;
+
 @RunWith(Parameterized.class)
 public class MutableIndexIT extends ParallelStatsDisabledIT {
     
     protected final boolean localIndex;
     private final String tableDDLOptions;
-	
+    
     public MutableIndexIT(Boolean localIndex, String txProvider, Boolean columnEncoded) {
-		this.localIndex = localIndex;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (txProvider != null) {
-			optionBuilder.append("TRANSACTIONAL=true," + PhoenixDatabaseMetaData.TRANSACTION_PROVIDER + "='" + txProvider + "'");
-		}
-		if (!columnEncoded) {
+        this.localIndex = localIndex;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (txProvider != null) {
+            optionBuilder.append("TRANSACTIONAL=true," + PhoenixDatabaseMetaData.TRANSACTION_PROVIDER + "='" + txProvider + "'");
+        }
+        if (!columnEncoded) {
             if (optionBuilder.length()!=0)
                 optionBuilder.append(",");
             optionBuilder.append("COLUMN_ENCODED_BYTES=0");
         }
-		this.tableDDLOptions = optionBuilder.toString();
-	}
+        this.tableDDLOptions = optionBuilder.toString();
+    }
     
     private static Connection getConnection(Properties props) throws SQLException {
         props.setProperty(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(1));
@@ -106,7 +106,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
         return getConnection(props);
     }
     
-	@Parameters(name="MutableIndexIT_localIndex={0},transactional={1},columnEncoded={2}") // name is used by failsafe as file name in reports
+    @Parameters(name="MutableIndexIT_localIndex={0},transactional={1},columnEncoded={2}") // name is used by failsafe as file name in reports
     public static Collection<Object[]> data() {
         return Arrays.asList(new Object[][] { 
                 { false, null, false }, { false, null, true },
@@ -121,16 +121,16 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     @Test
     public void testCoveredColumnUpdates() throws Exception {
         try (Connection conn = getConnection()) {
-	        conn.setAutoCommit(false);
-			String tableName = "TBL_" + generateUniqueName();
-			String indexName = "IDX_" + generateUniqueName();
-			String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-			String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            conn.setAutoCommit(false);
+            String tableName = "TBL_" + generateUniqueName();
+            String indexName = "IDX_" + generateUniqueName();
+            String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+            String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 
-			TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
+            TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName);
             conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName 
-            		+ " (char_col1 ASC, int_col1 ASC) INCLUDE (long_col1, long_col2)");
+                    + " (char_col1 ASC, int_col1 ASC) INCLUDE (long_col1, long_col2)");
             
             String query = "SELECT char_col1, int_col1, long_col2 from " + fullTableName;
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
@@ -203,7 +203,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
                 query = "SELECT b.* from " + fullTableName + " where int_col1 = 4";
                 rs = conn.createStatement().executeQuery("EXPLAIN " + query);
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName +" [1]\n" +
-                		"    SERVER FILTER BY TO_INTEGER(\"INT_COL1\") = 4\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        "    SERVER FILTER BY TO_INTEGER(\"INT_COL1\") = 4\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
                 rs = conn.createStatement().executeQuery(query);
                 assertTrue(rs.next());
                 assertEquals("varchar_b", rs.getString(1));
@@ -218,219 +218,219 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCoveredColumns() throws Exception {
-		String tableName = "TBL_" + generateUniqueName();
-		String indexName = "IDX_" + generateUniqueName();
-		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = getConnection()) {
 
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        conn.createStatement().execute("CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
-	        stmt.setString(1,"a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("a",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
-	        stmt.setString(1,"a");
-	        stmt.setString(2, null);
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("a",rs.getString(2));
-	        assertNull(rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
-	        }
-	
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertNull(rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
-	        stmt.setString(1,"a");
-	        stmt.setString(2,"3");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("3",rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
-	        stmt.setString(1,"a");
-	        stmt.setString(2,"4");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("4",rs.getString(3));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            
+            conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
+            stmt.setString(1,"a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("a",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertFalse(rs.next());
+    
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
+            stmt.setString(1,"a");
+            stmt.setString(2, null);
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("a",rs.getString(2));
+            assertNull(rs.getString(3));
+            assertFalse(rs.next());
+    
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
+            }
+    
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertNull(rs.getString(3));
+            assertFalse(rs.next());
+    
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
+            stmt.setString(1,"a");
+            stmt.setString(2,"3");
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
+            }
+            
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("3",rs.getString(3));
+            assertFalse(rs.next());
+    
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
+            stmt.setString(1,"a");
+            stmt.setString(2,"4");
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\nCLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));            
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
+            }
+            
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("4",rs.getString(3));
+            assertFalse(rs.next());
         }
     }
 
     @Test
     public void testCompoundIndexKey() throws Exception {
-		String tableName = "TBL_" + generateUniqueName();
-		String indexName = "IDX_" + generateUniqueName();
-		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = getConnection()) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute("CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v1, v2)");
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        // load some data into the table
-	        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
-	        stmt.setString(1,"a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("a",rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
-	        stmt.setString(1,"a");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, null);
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("y",rs.getString(1));
-	        assertNull(rs.getString(2));
-	        assertEquals("a",rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\n"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY\n"
-	                    + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
-	                       + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        //make sure the data table looks like what we expect
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertNull(rs.getString(3));
-	        assertFalse(rs.next());
-	        
-	        // Upsert new row with null leading index column
-	        stmt.setString(1,"b");
-	        stmt.setString(2, null);
-	        stmt.setString(3, "3");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(null,rs.getString(1));
-	        assertEquals("3",rs.getString(2));
-	        assertEquals("b",rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("y",rs.getString(1));
-	        assertNull(rs.getString(2));
-	        assertEquals("a",rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        // Update row with null leading index column to have a value
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?)");
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "z");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("y",rs.getString(1));
-	        assertNull(rs.getString(2));
-	        assertEquals("a",rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z",rs.getString(1));
-	        assertEquals("3",rs.getString(2));
-	        assertEquals("b",rs.getString(3));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            // make sure that the tables are empty, but reachable
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v1, v2)");
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            // load some data into the table
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
+            stmt.setString(1,"a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("a",rs.getString(3));
+            assertFalse(rs.next());
+    
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
+            stmt.setString(1,"a");
+            stmt.setString(2, "y");
+            stmt.setString(3, null);
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("y",rs.getString(1));
+            assertNull(rs.getString(2));
+            assertEquals("a",rs.getString(3));
+            assertFalse(rs.next());
+    
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\n"
+                        + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
+                           + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+            //make sure the data table looks like what we expect
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertNull(rs.getString(3));
+            assertFalse(rs.next());
+            
+            // Upsert new row with null leading index column
+            stmt.setString(1,"b");
+            stmt.setString(2, null);
+            stmt.setString(3, "3");
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(null,rs.getString(1));
+            assertEquals("3",rs.getString(2));
+            assertEquals("b",rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("y",rs.getString(1));
+            assertNull(rs.getString(2));
+            assertEquals("a",rs.getString(3));
+            assertFalse(rs.next());
+    
+            // Update row with null leading index column to have a value
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?)");
+            stmt.setString(1,"b");
+            stmt.setString(2, "z");
+            stmt.execute();
+            conn.commit();
+            
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("y",rs.getString(1));
+            assertNull(rs.getString(2));
+            assertEquals("a",rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z",rs.getString(1));
+            assertEquals("3",rs.getString(2));
+            assertEquals("b",rs.getString(3));
+            assertFalse(rs.next());
         }
 
     }
@@ -445,166 +445,166 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
      */
     @Test
     public void testMultipleUpdatesToSingleRow() throws Exception {
-		String tableName = "TBL_" + generateUniqueName();
-		String indexName = "IDX_" + generateUniqueName();
-		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = getConnection()) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + fullTableName
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v1, v2)");
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	    
-	        // load some data into the table
-	        PreparedStatement stmt =
-	            conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        // make sure the index is working as expected
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x", rs.getString(1));
-	        assertEquals("1", rs.getString(2));
-	        assertEquals("a", rs.getString(3));
-	        assertFalse(rs.next());
-	      
-	        // do multiple updates to the same row, in the same batch
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, null);
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertNull(rs.getString(2));
-	        assertEquals("a", rs.getString(3));
-	        assertFalse(rs.next());
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\n"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY\n"
-	                    + "CLIENT MERGE SORT",
-	                QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY",
-	                QueryUtil.getExplainPlan(rs));
-	        }
-	    
-	        // check that the data table matches as expected
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("a", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertNull(rs.getString(3));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            // make sure that the tables are empty, but reachable
+            conn.createStatement().execute(
+              "CREATE TABLE " + fullTableName
+                  + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            conn.createStatement().execute("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v1, v2)");
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+        
+            // load some data into the table
+            PreparedStatement stmt =
+                conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            conn.commit();
+            
+            // make sure the index is working as expected
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x", rs.getString(1));
+            assertEquals("1", rs.getString(2));
+            assertEquals("a", rs.getString(3));
+            assertFalse(rs.next());
+          
+            // do multiple updates to the same row, in the same batch
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k,v2) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, null);
+            stmt.execute();
+            conn.commit();
+        
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertNull(rs.getString(2));
+            assertEquals("a", rs.getString(3));
+            assertFalse(rs.next());
+        
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName+" [1]\n"
+                        + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                        + "CLIENT MERGE SORT",
+                    QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
+                        + "    SERVER FILTER BY FIRST KEY ONLY",
+                    QueryUtil.getExplainPlan(rs));
+            }
+        
+            // check that the data table matches as expected
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("a", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertNull(rs.getString(3));
+            assertFalse(rs.next());
         }
     }
     
     @Test
     public void testUpsertingNullForIndexedColumns() throws Exception {
-		String tableName = "TBL_" + generateUniqueName();
-		String indexName = "IDX_" + generateUniqueName();
-		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-        String testTableName = tableName + "_" + System.currentTimeMillis();
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        String testTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName + "_" + System.currentTimeMillis());
         try (Connection conn = getConnection()) {
-	        conn.setAutoCommit(false);
-	        ResultSet rs;
-    		Statement stmt = conn.createStatement();
-    		stmt.execute("CREATE TABLE " + testTableName + "(v1 VARCHAR PRIMARY KEY, v2 DOUBLE, v3 VARCHAR) "+tableDDLOptions);
-    		stmt.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + testTableName + "  (v2) INCLUDE(v3)");
-    		
-    		//create a row with value null for indexed column v2
-    		stmt.executeUpdate("upsert into " + testTableName + " values('cc1', null, 'abc')");
-    		conn.commit();
-    		
-    		//assert values in index table 
-    		rs = stmt.executeQuery("select * from " + fullIndexName);
-    		assertTrue(rs.next());
-    		assertEquals(0, Doubles.compare(0, rs.getDouble(1)));
-    		assertTrue(rs.wasNull());
-    		assertEquals("cc1", rs.getString(2));
-    		assertEquals("abc", rs.getString(3));
-    		assertFalse(rs.next());
-    		
-    		//assert values in data table
-    		rs = stmt.executeQuery("select v1, v2, v3 from " + testTableName);
-    		assertTrue(rs.next());
-    		assertEquals("cc1", rs.getString(1));
-    		assertEquals(0, Doubles.compare(0, rs.getDouble(2)));
-    		assertTrue(rs.wasNull());
-    		assertEquals("abc", rs.getString(3));
-    		assertFalse(rs.next());
-    		
-    		//update the previously null value for indexed column v2 to a non-null value 1.23
-    		stmt.executeUpdate("upsert into " + testTableName + " values('cc1', 1.23, 'abc')");
-    		conn.commit();
-    		
-    		//assert values in data table
-    		rs = stmt.executeQuery("select /*+ NO_INDEX */ v1, v2, v3 from " + testTableName);
-    		assertTrue(rs.next());
-    		assertEquals("cc1", rs.getString(1));
-    		assertEquals(0, Doubles.compare(1.23, rs.getDouble(2)));
-    		assertEquals("abc", rs.getString(3));
-    		assertFalse(rs.next());
-    		
-    		//assert values in index table 
-    		rs = stmt.executeQuery("select * from " + indexName);
-    		assertTrue(rs.next());
-    		assertEquals(0, Doubles.compare(1.23, rs.getDouble(1)));
-    		assertEquals("cc1", rs.getString(2));
-    		assertEquals("abc", rs.getString(3));
-    		assertFalse(rs.next());
-    		
-    		//update the value for indexed column v2 back to null
-    		stmt.executeUpdate("upsert into " + testTableName + " values('cc1', null, 'abc')");
-    		conn.commit();
-    		
-    		//assert values in index table 
-    		rs = stmt.executeQuery("select * from " + indexName);
-    		assertTrue(rs.next());
-    		assertEquals(0, Doubles.compare(0, rs.getDouble(1)));
-    		assertTrue(rs.wasNull());
-    		assertEquals("cc1", rs.getString(2));
-    		assertEquals("abc", rs.getString(3));
-    		assertFalse(rs.next());
-    		
-    		//assert values in data table
-    		rs = stmt.executeQuery("select v1, v2, v3 from " + testTableName);
-    		assertTrue(rs.next());
-    		assertEquals("cc1", rs.getString(1));
-    		assertEquals(0, Doubles.compare(0, rs.getDouble(2)));
-    		assertEquals("abc", rs.getString(3));
-    		assertFalse(rs.next());
-    	} 
+            conn.setAutoCommit(false);
+            ResultSet rs;
+            Statement stmt = conn.createStatement();
+            stmt.execute("CREATE TABLE " + testTableName + "(v1 VARCHAR PRIMARY KEY, v2 DOUBLE, v3 VARCHAR) "+tableDDLOptions);
+            stmt.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + testTableName + "  (v2) INCLUDE(v3)");
+            
+            //create a row with value null for indexed column v2
+            stmt.executeUpdate("upsert into " + testTableName + " values('cc1', null, 'abc')");
+            conn.commit();
+            
+            //assert values in index table 
+            rs = stmt.executeQuery("select * from " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(0, Doubles.compare(0, rs.getDouble(1)));
+            assertTrue(rs.wasNull());
+            assertEquals("cc1", rs.getString(2));
+            assertEquals("abc", rs.getString(3));
+            assertFalse(rs.next());
+            
+            //assert values in data table
+            rs = stmt.executeQuery("select v1, v2, v3 from " + testTableName);
+            assertTrue(rs.next());
+            assertEquals("cc1", rs.getString(1));
+            assertEquals(0, Doubles.compare(0, rs.getDouble(2)));
+            assertTrue(rs.wasNull());
+            assertEquals("abc", rs.getString(3));
+            assertFalse(rs.next());
+            
+            //update the previously null value for indexed column v2 to a non-null value 1.23
+            stmt.executeUpdate("upsert into " + testTableName + " values('cc1', 1.23, 'abc')");
+            conn.commit();
+            
+            //assert values in data table
+            rs = stmt.executeQuery("select /*+ NO_INDEX */ v1, v2, v3 from " + testTableName);
+            assertTrue(rs.next());
+            assertEquals("cc1", rs.getString(1));
+            assertEquals(0, Doubles.compare(1.23, rs.getDouble(2)));
+            assertEquals("abc", rs.getString(3));
+            assertFalse(rs.next());
+            
+            //assert values in index table 
+            rs = stmt.executeQuery("select * from " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(0, Doubles.compare(1.23, rs.getDouble(1)));
+            assertEquals("cc1", rs.getString(2));
+            assertEquals("abc", rs.getString(3));
+            assertFalse(rs.next());
+            
+            //update the value for indexed column v2 back to null
+            stmt.executeUpdate("upsert into " + testTableName + " values('cc1', null, 'abc')");
+            conn.commit();
+            
+            //assert values in index table 
+            rs = stmt.executeQuery("select * from " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(0, Doubles.compare(0, rs.getDouble(1)));
+            assertTrue(rs.wasNull());
+            assertEquals("cc1", rs.getString(2));
+            assertEquals("abc", rs.getString(3));
+            assertFalse(rs.next());
+            
+            //assert values in data table
+            rs = stmt.executeQuery("select v1, v2, v3 from " + testTableName);
+            assertTrue(rs.next());
+            assertEquals("cc1", rs.getString(1));
+            assertEquals(0, Doubles.compare(0, rs.getDouble(2)));
+            assertEquals("abc", rs.getString(3));
+            assertFalse(rs.next());
+        } 
     }
     
-	
+    
     private void assertImmutableRows(Connection conn, String fullTableName, boolean expectedValue) throws SQLException {
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
         assertEquals(expectedValue, pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isImmutableRows());
@@ -614,25 +614,25 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     public void testAlterTableWithImmutability() throws Exception {
         String query;
         ResultSet rs;
-		String tableName = "TBL_" + generateUniqueName();
-		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String tableName = "TBL_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 
         try (Connection conn = getConnection()) {
-	        conn.setAutoCommit(false);
-	        conn.createStatement().execute(
-	            "CREATE TABLE " + fullTableName +" (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) " + tableDDLOptions);
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        assertImmutableRows(conn,fullTableName, false);
-	        conn.createStatement().execute("ALTER TABLE " + fullTableName +" SET IMMUTABLE_ROWS=true");
-	        assertImmutableRows(conn,fullTableName, true);
-	        
-	        
-	        conn.createStatement().execute("ALTER TABLE " + fullTableName +" SET immutable_rows=false");
-	        assertImmutableRows(conn,fullTableName, false);
+            conn.setAutoCommit(false);
+            conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName +" (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) " + tableDDLOptions);
+            
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            assertImmutableRows(conn,fullTableName, false);
+            conn.createStatement().execute("ALTER TABLE " + fullTableName +" SET IMMUTABLE_ROWS=true");
+            assertImmutableRows(conn,fullTableName, true);
+            
+            
+            conn.createStatement().execute("ALTER TABLE " + fullTableName +" SET immutable_rows=false");
+            assertImmutableRows(conn,fullTableName, false);
         }
     }
 
@@ -652,9 +652,9 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     public void testIndexHalfStoreFileReader() throws Exception {
         Connection conn1 = getConnection();
         ConnectionQueryServices connectionQueryServices = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES);
-		HBaseAdmin admin = connectionQueryServices.getAdmin();
-		String tableName = "TBL_" + generateUniqueName();
-		String indexName = "IDX_" + generateUniqueName();
+        HBaseAdmin admin = connectionQueryServices.getAdmin();
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
         createBaseTable(conn1, tableName, "('e')");
         conn1.createStatement().execute("CREATE "+(localIndex?"LOCAL":"")+" INDEX " + indexName + " ON " + tableName + "(v1)" + (localIndex?"":" SPLIT ON ('e')"));
         conn1.createStatement().execute("UPSERT INTO "+tableName+" values('b',1,2,4,'z')");
@@ -791,10 +791,10 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     
   @Test
   public void testTenantSpecificConnection() throws Exception {
-	  String tableName = "TBL_" + generateUniqueName();
-	  String indexName = "IDX_" + generateUniqueName();
-	  String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-	  Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+      String tableName = "TBL_" + generateUniqueName();
+      String indexName = "IDX_" + generateUniqueName();
+      String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+      Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
       try (Connection conn = getConnection()) {
           conn.setAutoCommit(false);
           // create data table
@@ -910,6 +910,48 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
       }
   }
 
+
+  @Test
+  public void testUpsertingDeletedRowShouldGiveProperDataWithIndexes() throws Exception {
+      testUpsertingDeletedRowShouldGiveProperDataWithIndexes(false);
+  }
+
+  @Test
+  public void testUpsertingDeletedRowShouldGiveProperDataWithMultiCFIndexes() throws Exception {
+      testUpsertingDeletedRowShouldGiveProperDataWithIndexes(true);
+  }
+
+  private void testUpsertingDeletedRowShouldGiveProperDataWithIndexes(boolean multiCf) throws Exception {
+      String tableName = "TBL_" + generateUniqueName();
+      String indexName = "IDX_" + generateUniqueName();
+      String columnFamily1 = "cf1";
+      String columnFamily2 = "cf2";
+      String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+      String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+      try (Connection conn = getConnection()) {
+            conn.createStatement().execute(
+                "create table " + fullTableName + " (id integer primary key, "
+                        + (multiCf ? columnFamily1 : "") + "f float, "
+                        + (multiCf ? columnFamily2 : "") + "s varchar)" + tableDDLOptions);
+            conn.createStatement().execute(
+                "create index " + indexName + " on " + fullTableName + " ("
+                        + (multiCf ? columnFamily1 : "") + "f) include ("+(multiCf ? columnFamily2 : "") +"s)");
+            conn.createStatement().execute(
+                "upsert into " + fullTableName + " values (1, 0.5, 'foo')");
+          conn.commit();
+          conn.createStatement().execute("delete from  " + fullTableName + " where id = 1");
+          conn.commit();
+            conn.createStatement().execute(
+                "upsert into  " + fullTableName + " values (1, 0.5, 'foo')");
+          conn.commit();
+          ResultSet rs = conn.createStatement().executeQuery("select * from "+fullIndexName);
+          assertTrue(rs.next());
+          assertEquals(1, rs.getInt(2));
+          assertEquals(0.5F, rs.getFloat(1), 0.0);
+          assertEquals("foo", rs.getString(3));
+      } 
+  }
+
 private void upsertRow(String dml, Connection tenantConn, int i) throws SQLException {
     PreparedStatement stmt = tenantConn.prepareStatement(dml);
       stmt.setString(1, "00000000000000" + String.valueOf(i));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 8ffd798..3ce6ae5 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
@@ -39,7 +39,7 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.QueryPlan;
-import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.end2end.SplitSystemCatalogIT;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
@@ -59,7 +59,7 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
-public class ViewIndexIT extends ParallelStatsDisabledIT {
+public class ViewIndexIT extends SplitSystemCatalogIT {
     private boolean isNamespaceMapped;
 
     @Parameters(name = "ViewIndexIT_isNamespaceMapped={0}") // name is used by failsafe as file name in reports
@@ -108,10 +108,11 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
     public void testDeleteViewIndexSequences() throws Exception {
         String schemaName = generateUniqueName();
         String tableName = generateUniqueName();
+        String viewSchemaName = generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
         String indexName = "IND_" + generateUniqueName();
         String viewName = "VIEW_" + generateUniqueName();
-        String fullViewName = SchemaUtil.getTableName(schemaName, viewName);
+        String fullViewName = SchemaUtil.getTableName(viewSchemaName, viewName);
 
         createBaseTable(schemaName, tableName, false, null, null);
         Connection conn1 = getConnection();
@@ -138,13 +139,12 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testMultiTenantViewLocalIndex() throws Exception {
-        String schemaName = generateUniqueName();
-        String tableName =  generateUniqueName();
-        String indexName = "IND_" + generateUniqueName();
-        String viewName = "VIEW_" + generateUniqueName();
-        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String tableName = generateUniqueName();
+		String indexName = "IND_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1, tableName);
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         
-        createBaseTable(schemaName, tableName, true, null, null);
+        createBaseTable(SCHEMA1, tableName, true, null, null);
         Connection conn = DriverManager.getConnection(getUrl());
         PreparedStatement stmt = conn.prepareStatement(
                 "UPSERT INTO " + fullTableName
@@ -178,14 +178,14 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
         Properties props  = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         props.setProperty("TenantId", "10");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
-        conn1.createStatement().execute("CREATE VIEW " + viewName
+        conn1.createStatement().execute("CREATE VIEW " + fullViewName
                 + " AS select * from " + fullTableName);
         conn1.createStatement().execute("CREATE LOCAL INDEX "
                 + indexName + " ON "
-                + viewName + "(v2)");
+                + fullViewName + "(v2)");
         conn1.commit();
         
-        String sql = "SELECT * FROM " + viewName + " WHERE v2 = 100";
+        String sql = "SELECT * FROM " + fullViewName + " WHERE v2 = 100";
         ResultSet rs = conn1.prepareStatement("EXPLAIN " + sql).executeQuery();
         assertEquals(
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + SchemaUtil.getPhysicalTableName(Bytes.toBytes(fullTableName), isNamespaceMapped) + " [1,'10',100]\n" +
@@ -208,7 +208,8 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
 
         // Confirm that when view index used, the GUIDE_POSTS_WIDTH from the data physical table
         // was used
-        sql = "SELECT * FROM " + viewName + " WHERE v2 >= 100";
+        sql = "SELECT * FROM " + fullViewName + " WHERE v2 >= 100";
+        rs = conn1.prepareStatement("EXPLAIN " + sql).executeQuery();
         stmt = conn1.prepareStatement(sql);
         stmt.executeQuery();
         QueryPlan plan = stmt.unwrap(PhoenixStatement.class).getQueryPlan();
@@ -218,9 +219,10 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCreatingIndexOnGlobalView() throws Exception {
-        String baseTable =  generateUniqueName();
-        String globalView = generateUniqueName();
+        String baseTable =  SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String globalView = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
         String globalViewIdx =  generateUniqueName();
+        String fullIndexName = SchemaUtil.getTableName(SCHEMA2, globalViewIdx);
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute("CREATE IMMUTABLE TABLE " + baseTable + " (TENANT_ID CHAR(15) NOT NULL, PK2 DATE NOT NULL, PK3 INTEGER NOT NULL, KV1 VARCHAR, KV2 VARCHAR, KV3 CHAR(15) CONSTRAINT PK PRIMARY KEY(TENANT_ID, PK2 ROW_TIMESTAMP, PK3)) MULTI_TENANT=true");
             conn.createStatement().execute("CREATE VIEW " + globalView + " AS SELECT * FROM " + baseTable);
@@ -264,7 +266,7 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
             stmt.setString(2, "KV3");
             ResultSet rs = stmt.executeQuery();
             QueryPlan plan = stmt.unwrap(PhoenixStatement.class).getQueryPlan();
-            assertTrue(plan.getTableRef().getTable().getName().getString().equals(globalViewIdx));
+            assertEquals(fullIndexName, plan.getTableRef().getTable().getName().getString());
             assertTrue(rs.next());
             assertEquals("KV1", rs.getString(1));
             assertFalse(rs.next());
@@ -287,7 +289,7 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
             stmt.setString(2, "KV3");
             rs = stmt.executeQuery();
             plan = stmt.unwrap(PhoenixStatement.class).getQueryPlan();
-            assertTrue(plan.getTableRef().getTable().getName().getString().equals(globalViewIdx));
+            assertEquals(fullIndexName, plan.getTableRef().getTable().getName().getString());
             assertEquals(6, plan.getSplits().size());
         }
     }
@@ -319,14 +321,17 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
     @Test
     public void testUpdateOnTenantViewWithGlobalView() throws Exception {
         Connection conn = getConnection();
-        String baseSchemaName = "PLATFORM_ENTITY";
+        String baseSchemaName = generateUniqueName();
+        String viewSchemaName = generateUniqueName();
+        String tsViewSchemaName = generateUniqueName();
         String baseTableName = generateUniqueName();
         String baseFullName = SchemaUtil.getTableName(baseSchemaName, baseTableName);
         String viewTableName = "V_" + generateUniqueName();
-        String viewFullName = SchemaUtil.getTableName(baseSchemaName, viewTableName);
+        String viewFullName = SchemaUtil.getTableName(viewSchemaName, viewTableName);
         String indexName = "I_" + generateUniqueName();
         String tsViewTableName = "TSV_" + generateUniqueName();
-        String tsViewFullName = SchemaUtil.getTableName(baseSchemaName, tsViewTableName);
+        String tsViewFullName = SchemaUtil.getTableName(tsViewSchemaName, tsViewTableName);
+        String tenantId = "tenant1";
         try {
             if (isNamespaceMapped) {
                 conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + baseSchemaName);
@@ -348,7 +353,7 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
                     "CREATE INDEX " + indexName + " \n" + "ON " + viewFullName + " (TEXT1 DESC, INT1)\n"
                             + "INCLUDE (CREATED_BY, DOUBLE1, IS_BOOLEAN, CREATED_DATE)");
             Properties tsProps = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            tsProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "123451234512345");
+            tsProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
             Connection tsConn = DriverManager.getConnection(getUrl(), tsProps);
             tsConn.createStatement().execute("CREATE VIEW " + tsViewFullName + " AS SELECT * FROM " + viewFullName);
             tsConn.createStatement().execute("UPSERT INTO " + tsViewFullName + "(INT1,DOUBLE1,IS_BOOLEAN,TEXT1) VALUES (1,1.0, true, 'a')");
@@ -430,9 +435,10 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
         Properties props = new Properties();
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.setAutoCommit(true);
-        String tableName=generateUniqueName();
-        String viewName=generateUniqueName();
-        String indexName=generateUniqueName();
+        String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String indexName = generateUniqueName();
+        String fullIndexName = SchemaUtil.getTableName(SCHEMA2, indexName);
         conn1.createStatement().execute(
           "CREATE TABLE "+tableName+" (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) UPDATE_CACHE_FREQUENCY=1000000");
         conn1.createStatement().execute("upsert into "+tableName+" values ('row1', 'value1', 'key1')");
@@ -440,19 +446,21 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
           "CREATE VIEW "+viewName+" (v3 VARCHAR, v4 VARCHAR) AS SELECT * FROM "+tableName+" WHERE v1 = 'value1'");
         conn1.createStatement().execute("CREATE INDEX " + indexName + " ON " + viewName + "(v3)" + (includeColumns ? " INCLUDE(v4)" : ""));
         PhoenixStatement stmt = conn1.createStatement().unwrap(PhoenixStatement.class);
-        ResultSet rs = stmt.executeQuery("SELECT /*+ INDEX(" + viewName + " " + indexName + ") */ v1 FROM " + viewName + " WHERE v3 = 'foo' ORDER BY v4");
+        ResultSet rs = stmt.executeQuery("SELECT /*+ INDEX(" + viewName + " " + fullIndexName + ") */ v1 FROM " + viewName + " WHERE v3 = 'foo' ORDER BY v4");
         assertFalse(rs.next());
-        assertEquals(indexName, stmt.getQueryPlan().getContext().getCurrentTable().getTable().getName().getString());
+        assertEquals(fullIndexName, 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();
-
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String indexName=generateUniqueName();
             c.setAutoCommit(true);
+            if (isNamespaceMapped) {
+                c.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + SCHEMA1);
+            }
             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);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index e5b57e3..feb0ce4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@ -28,6 +28,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -60,6 +61,7 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -286,7 +288,7 @@ public class PartialCommitIT extends BaseOwnClusterIT {
         PhoenixConnection phxCon = new PhoenixConnection(con.unwrap(PhoenixConnection.class));
         final Map<TableRef, MultiRowMutationState> mutations = Maps.newTreeMap(new TableRefComparator());
         // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
-        return new PhoenixConnection(phxCon, null) {
+        return new PhoenixConnection(phxCon, (MutationState)null) {
             @Override
             protected MutationState newMutationState(int maxSize, int maxSizeBytes) {
                 return new MutationState(maxSize, maxSizeBytes, this, mutations, false, null);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
index 776e300..c11d5ef 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
@@ -32,7 +32,6 @@ import org.apache.phoenix.mapreduce.util.ConnectionUtil;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -69,9 +68,12 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
   private static final String DROP_TENANT_VIEW_SQL = "DROP VIEW IF EXISTS " + TENANT_VIEW_NAME;
   private static final String DROP_NONTENANT_VIEW_SQL = "DROP VIEW IF EXISTS " + NONTENANT_VIEW_NAME;
   private static PTable catalogTable;
+  private static PTable childLinkTable;
   private static WALKey walKey = null;
   private static TableName systemCatalogTableName =
       TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+  private static TableName systemChildLinkTableName =
+	      TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME);
 
 
   @BeforeClass
@@ -85,6 +87,7 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
       ensureTableCreated(getUrl(), TestUtil.ENTITY_HISTORY_TABLE_NAME);
       connection.createStatement().execute(CREATE_TENANT_VIEW_SQL);
       catalogTable = PhoenixRuntime.getTable(connection, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+      childLinkTable = PhoenixRuntime.getTable(connection, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME);
       walKey = new WALKey(REGION, TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME), 0, 0, uuid);
     };
     Assert.assertNotNull(catalogTable);
@@ -123,63 +126,36 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
 
     //now create WAL.Entry objects that refer to cells in those view rows in System.Catalog
 
-    Get tenantViewGet = getTenantViewGet(catalogTable, TENANT_BYTES, TENANT_VIEW_NAME);
-    Get nonTenantViewGet = getTenantViewGet(catalogTable,
-        DEFAULT_TENANT_BYTES, NONTENANT_VIEW_NAME);
+    Get tenantGet = getGet(catalogTable, TENANT_BYTES, TENANT_VIEW_NAME);
+    Get nonTenantGet = getGet(catalogTable, DEFAULT_TENANT_BYTES, NONTENANT_VIEW_NAME);
 
-    Get tenantLinkGet = getParentChildLinkGet(catalogTable, TENANT_BYTES, TENANT_VIEW_NAME);
-    Get nonTenantLinkGet = getParentChildLinkGet(catalogTable,
-        DEFAULT_TENANT_BYTES, NONTENANT_VIEW_NAME);
-
-    WAL.Entry nonTenantViewEntry = getEntry(systemCatalogTableName, nonTenantViewGet);
-    WAL.Entry tenantViewEntry = getEntry(systemCatalogTableName, tenantViewGet);
-
-    WAL.Entry nonTenantLinkEntry = getEntry(systemCatalogTableName, nonTenantLinkGet);
-    WAL.Entry tenantLinkEntry = getEntry(systemCatalogTableName, tenantLinkGet);
+    WAL.Entry nonTenantEntry = getEntry(systemCatalogTableName, nonTenantGet);
+    WAL.Entry tenantEntry = getEntry(systemCatalogTableName, tenantGet);
 
     //verify that the tenant view WAL.Entry passes the filter and the non-tenant view does not
     SystemCatalogWALEntryFilter filter = new SystemCatalogWALEntryFilter();
-    Assert.assertNull(filter.filter(nonTenantViewEntry));
-    WAL.Entry filteredTenantEntry = filter.filter(tenantViewEntry);
+    Assert.assertNull(filter.filter(nonTenantEntry));
+    WAL.Entry filteredTenantEntry = filter.filter(tenantEntry);
     Assert.assertNotNull("Tenant view was filtered when it shouldn't be!", filteredTenantEntry);
-    Assert.assertEquals(tenantViewEntry.getEdit().size(),
-        filter.filter(tenantViewEntry).getEdit().size());
+    Assert.assertEquals(tenantEntry.getEdit().size(),
+        filter.filter(tenantEntry).getEdit().size());
 
     //now check that a WAL.Entry with cells from both a tenant and a non-tenant
     //catalog row only allow the tenant cells through
     WALEdit comboEdit = new WALEdit();
-    comboEdit.getCells().addAll(nonTenantViewEntry.getEdit().getCells());
-    comboEdit.getCells().addAll(tenantViewEntry.getEdit().getCells());
+    comboEdit.getCells().addAll(nonTenantEntry.getEdit().getCells());
+    comboEdit.getCells().addAll(tenantEntry.getEdit().getCells());
     WAL.Entry comboEntry = new WAL.Entry(walKey, comboEdit);
 
-    Assert.assertEquals(tenantViewEntry.getEdit().size() + nonTenantViewEntry.getEdit().size()
+    Assert.assertEquals(tenantEntry.getEdit().size() + nonTenantEntry.getEdit().size()
         , comboEntry.getEdit().size());
-    Assert.assertEquals(tenantViewEntry.getEdit().size(),
+    Assert.assertEquals(tenantEntry.getEdit().size(),
         filter.filter(comboEntry).getEdit().size());
-
-    //now check that the parent-child links (which have the tenant_id of the view's parent,
-    // but are a part of the view's metadata) are migrated in the tenant case
-    // but not the non-tenant. The view's tenant_id is in th System.Catalog.COLUMN_NAME field
-
-    Assert.assertNull("Non-tenant parent-child link was not filtered " +
-        "when it should be!", filter.filter(nonTenantLinkEntry));
-    Assert.assertNotNull("Tenant parent-child link was filtered when it should not be!",
-        filter.filter(tenantLinkEntry));
-    Assert.assertEquals(tenantLinkEntry.getEdit().size(),
-        filter.filter(tenantLinkEntry).getEdit().size());
-    //add the parent-child link to the tenant view WAL entry,
-    //since they'll usually be together and they both need to
-    //be replicated
-
-    tenantViewEntry.getEdit().getCells().addAll(tenantLinkEntry.getEdit().getCells());
-    Assert.assertEquals(tenantViewEntry.getEdit().size(), tenantViewEntry.getEdit().size());
-
-
   }
 
-  public Get getTenantViewGet(PTable catalogTable, byte[] tenantBytes, String viewName) {
+  public Get getGet(PTable catalogTable, byte[] tenantId, String viewName) {
     byte[][] tenantKeyParts = new byte[5][];
-    tenantKeyParts[0] = tenantBytes;
+    tenantKeyParts[0] = tenantId;
     tenantKeyParts[1] = Bytes.toBytes(SCHEMA_NAME.toUpperCase());
     tenantKeyParts[2] = Bytes.toBytes(viewName.toUpperCase());
     tenantKeyParts[3] = Bytes.toBytes(VIEW_COLUMN_NAME);
@@ -191,28 +167,6 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
     return new Get(key.copyBytes());
   }
 
-  public Get getParentChildLinkGet(PTable catalogTable, byte[] tenantBytes, String viewName) {
-    /* For parent-child link, the system.catalog key becomes
-      1. Parent tenant id
-      2. Parent Schema
-      3. Parent Table name
-      4. View tenant_id
-      5. Combined view SCHEMA.TABLENAME
-     */
-    byte[][] tenantKeyParts = new byte[5][];
-    tenantKeyParts[0] = null; //null tenant_id
-    tenantKeyParts[1] = null; //null parent schema
-    tenantKeyParts[2] = Bytes.toBytes(TestUtil.ENTITY_HISTORY_TABLE_NAME);
-    tenantKeyParts[3] = tenantBytes;
-    tenantKeyParts[4] = Bytes.toBytes(SchemaUtil.getTableName(SCHEMA_NAME.toUpperCase(), viewName.toUpperCase()));
-    ImmutableBytesWritable key = new ImmutableBytesWritable();
-    catalogTable.newKey(key, tenantKeyParts);
-    //the backing byte array of key might have extra space at the end.
-    // need to just slice "the good parts" which we do by calling copyBytes
-    return new Get(key.copyBytes());
-
-  }
-
   public WAL.Entry getEntry(TableName tableName, Get get) throws IOException {
     WAL.Entry entry = null;
     try(Connection conn = ConnectionFactory.createConnection(getUtility().getConfiguration())){
@@ -226,8 +180,7 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
           edit.add(c);
         }
       }
-      Assert.assertTrue("Didn't retrieve any cells from SYSTEM.CATALOG",
-          edit.getCells().size() > 0);
+      Assert.assertTrue("Didn't retrieve any cells from SYSTEM.CATALOG", edit.getCells().size() > 0);
       WALKey key = new WALKey(REGION, tableName, 0, 0, uuid);
       entry = new WAL.Entry(key, edit);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index e6a2f7d..2211d58 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -160,7 +161,9 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
 			conn.commit();
             int numUpsertRpcs = expectedRPCs[0];
 			// verify only 0 or 1 rpc to fetch table metadata, 
-            verify(connectionQueryServices, times(numUpsertRpcs)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
+            verify(connectionQueryServices, times(numUpsertRpcs)).getTable((PName) isNull(),
+                eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)),
+                anyLong(), anyLong(), eq(false), eq(false), (PTable)isNull());
             reset(connectionQueryServices);
             
             ResultSet rs = conn.createStatement().executeQuery(selectSql);
@@ -186,7 +189,9 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
             // for transactional tables : verify *only* one rpc occurs
 	        // for non-transactional, system tables : verify no rpc occurs
             int numRpcs = expectedRPCs[1]; 
-            verify(connectionQueryServices, times(numRpcs)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
+            verify(connectionQueryServices, times(numRpcs)).getTable((PName) isNull(),
+                eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)),
+                anyLong(), anyLong(), eq(false), eq(false), (PTable)isNull());
 		}
         finally {
         	conn.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnNameTrackingExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnNameTrackingExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnNameTrackingExpressionCompiler.java
new file mode 100644
index 0000000..f1bba07
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnNameTrackingExpressionCompiler.java
@@ -0,0 +1,46 @@
+/*
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
+
+import com.google.common.collect.Lists;
+
+public class ColumnNameTrackingExpressionCompiler extends StatelessTraverseAllParseNodeVisitor {
+
+    private List<String> dataColumnNames = Lists.newArrayListWithExpectedSize(10);
+
+    public void reset() {
+        this.getDataColumnNames().clear();
+    }
+
+    @Override
+    public Void visit(ColumnParseNode node) throws SQLException {
+        getDataColumnNames().add(node.getName());
+        return null;
+    }
+
+    public List<String> getDataColumnNames() {
+        return dataColumnNames;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index 4e5580a..d67899c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -204,7 +204,7 @@ public class CreateTableCompiler {
         };
     }
     
-    private static class ColumnTrackingExpressionCompiler extends ExpressionCompiler {
+    public static class ColumnTrackingExpressionCompiler extends ExpressionCompiler {
         private final BitSet isColumnReferenced;
         
         public ColumnTrackingExpressionCompiler(StatementContext context, BitSet isColumnReferenced) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 3faada7..4adc805 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
@@ -31,6 +31,9 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.AliasedNode;
@@ -272,7 +275,8 @@ public class FromCompiler {
             Expression sourceExpression = projector.getColumnProjector(column.getPosition()).getExpression();
             PColumnImpl projectedColumn = new PColumnImpl(column.getName(), column.getFamilyName(),
                     sourceExpression.getDataType(), sourceExpression.getMaxLength(), sourceExpression.getScale(), sourceExpression.isNullable(),
-                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes());
+                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes(),
+                column.getTimestamp());
             projectedColumns.add(projectedColumn);
         }
         PTable t = PTableImpl.makePTable(table, projectedColumns);
@@ -573,8 +577,9 @@ public class FromCompiler {
                 MetaDataMutationResult result = client.updateCache(tenantId, schemaName, tableName, alwaysHitServer);
                 timeStamp = TransactionUtil.getResolvedTimestamp(connection, result);
                 theTable = result.getTable();
+                MutationCode mutationCode = result.getMutationCode();
                 if (theTable == null) {
-                    throw new TableNotFoundException(schemaName, tableName, timeStamp);
+					throw new TableNotFoundException(schemaName, tableName, timeStamp);
                 }
             } else {
                 try {
@@ -710,7 +715,8 @@ public class FromCompiler {
                         familyName = PNameFactory.newName(family);
                     }
                     allcolumns.add(new PColumnImpl(name, familyName, dynColumn.getDataType(), dynColumn.getMaxLength(),
-                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true, Bytes.toBytes(dynColumn.getColumnDefName().getColumnName())));
+                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true, Bytes.toBytes(dynColumn.getColumnDefName().getColumnName()),
+                        HConstants.LATEST_TIMESTAMP));
                     position++;
                 }
                 theTable = PTableImpl.makePTable(theTable, allcolumns);
@@ -815,7 +821,8 @@ public class FromCompiler {
                 PName name = PNameFactory.newName(alias);
                 PColumnImpl column = new PColumnImpl(PNameFactory.newName(alias),
                         PNameFactory.newName(QueryConstants.DEFAULT_COLUMN_FAMILY),
-                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false, name.getBytes());
+                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false, name.getBytes(),
+                    HConstants.LATEST_TIMESTAMP);
                 columns.add(column);
             }
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
index e3ed110..5d7ace3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
@@ -85,7 +85,7 @@ public class ListJarsQueryPlan implements QueryPlan {
         PColumn column =
                 new PColumnImpl(colName, null,
                         PVarchar.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false, false, colName.getBytes());
+                        false, null, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
index 02aadc5..619ee24 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
@@ -82,7 +82,7 @@ public class TraceQueryPlan implements QueryPlan {
         PColumn column =
                 new PColumnImpl(PNameFactory.newName(MetricInfo.TRACE.columnName), null,
                         PLong.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false, false, colName.getBytes());
+                        false, null, false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 9ca92f9..c94634b 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
@@ -92,7 +92,7 @@ public class UnionCompiler {
                 UNION_FAMILY_NAME, targetTypes.get(i).getType(), targetTypes.get(i).getMaxLength(),
                 targetTypes.get(i).getScale(), colProj.getExpression().isNullable(), i,
                 targetTypes.get(i).getSortOrder(), 500, null, false,
-                colProj.getExpression().toString(), false, false, colName.getBytes());
+                colProj.getExpression().toString(), false, false, colName.getBytes(), HConstants.LATEST_TIMESTAMP);
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 c3cfa10..9d75bba 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
@@ -787,7 +787,7 @@ public class UpsertCompiler {
                 LinkedHashSet<PColumn> updateColumns = Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
                 updateColumns.add(new PColumnImpl(
                         table.getPKColumns().get(position).getName(), // Use first PK column name as we know it won't conflict with others
-                        null, PVarbinary.INSTANCE, null, null, false, position, SortOrder.getDefault(), 0, null, false, null, false, false, null));
+                        null, PVarbinary.INSTANCE, null, null, false, position, SortOrder.getDefault(), 0, null, false, null, false, false, null, table.getPKColumns().get(position).getTimestamp()));
                 position++;
                 for (Pair<ColumnName,ParseNode> columnPair : onDupKeyPairs) {
                     ColumnName colName = columnPair.getFirst();


[30/50] [abbrv] phoenix git commit: PHOENIX-4854 Make LoggingPhoenixResultSet idempotent when logging metrics

Posted by ja...@apache.org.
PHOENIX-4854 Make LoggingPhoenixResultSet idempotent when logging metrics


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

Branch: refs/heads/omid2
Commit: 7a2edb871884d38a4f3dd4472bb5b2ca139bf0f1
Parents: e979241
Author: Karan Mehta <ka...@gmail.com>
Authored: Mon Aug 20 10:12:37 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Mon Aug 20 10:45:04 2018 -0700

----------------------------------------------------------------------
 .../monitoring/PhoenixLoggingMetricsIT.java     | 49 +++++++++++---------
 .../phoenix/jdbc/LoggingPhoenixResultSet.java   | 15 ++++--
 2 files changed, 38 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2edb87/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
index 97b2c5d..7e56902 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
@@ -26,6 +26,7 @@ import org.junit.Test;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Map;
 
@@ -44,6 +45,8 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     private String tableName2;
     private LoggingPhoenixConnection loggedConn;
     private String loggedSql;
+    private int logOverAllReadRequestMetricsFuncCallCount;
+    private int logRequestReadMetricsFuncCallCount;
 
     @Before
     public void beforeTest() throws Exception {
@@ -69,17 +72,7 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     public void testPhoenixMetricsLoggedOnCommit() throws Exception {
         // run SELECT to verify read metrics are logged
         String query = "SELECT * FROM " + tableName1;
-        Statement stmt = loggedConn.createStatement();
-        ResultSet rs = stmt.executeQuery(query);
-        while (rs.next()) {
-        }
-        rs.close();
-        assertTrue("Read metrics for not found for " + tableName1,
-                requestReadMetricsMap.get(tableName1).size() > 0);
-        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
-
-        assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
-        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
+        verifyQueryLevelMetricsLogging(query);
 
         // run UPSERT SELECT to verify mutation metrics are logged
         String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
@@ -117,17 +110,7 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
     public void testPhoenixMetricsLoggedOnClose() throws Exception {
         // run SELECT to verify read metrics are logged
         String query = "SELECT * FROM " + tableName1;
-        Statement stmt = loggedConn.createStatement();
-        ResultSet rs = stmt.executeQuery(query);
-        while (rs.next()) {
-        }
-        rs.close();
-        assertTrue("Read metrics for not found for " + tableName1,
-                requestReadMetricsMap.get(tableName1).size() > 0);
-        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
-
-        assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
-        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
+        verifyQueryLevelMetricsLogging(query);
 
         // run UPSERT SELECT to verify mutation metrics are logged
         String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
@@ -151,6 +134,26 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
                 mutationReadMetricsMap.size() == 0);
     }
 
+    private void verifyQueryLevelMetricsLogging(String query) throws SQLException {
+        Statement stmt = loggedConn.createStatement();
+        ResultSet rs = stmt.executeQuery(query);
+        while (rs.next()) {
+        }
+        rs.close();
+        assertTrue("Read metrics for not found for " + tableName1,
+                requestReadMetricsMap.get(tableName1).size() > 0);
+        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
+        assertTrue(logOverAllReadRequestMetricsFuncCallCount == 1);
+        assertTrue(logRequestReadMetricsFuncCallCount == 1);
+
+        assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
+        assertTrue("Logged query doesn't match actual query", loggedSql.equals(query));
+
+        rs.close();
+        assertTrue(logOverAllReadRequestMetricsFuncCallCount == 1);
+        assertTrue(logRequestReadMetricsFuncCallCount == 1);
+    }
+
     void clearAllTestMetricMaps() {
         overAllQueryMetricsMap.clear();
         requestReadMetricsMap.clear();
@@ -165,6 +168,7 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
                     Map<MetricType, Long> overAllQueryMetrics, String sql) {
                 overAllQueryMetricsMap.putAll(overAllQueryMetrics);
                 loggedSql = sql;
+                logOverAllReadRequestMetricsFuncCallCount++;
             }
 
             @Override
@@ -172,6 +176,7 @@ public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
                     Map<String, Map<MetricType, Long>> requestReadMetrics, String sql) {
                 requestReadMetricsMap.putAll(requestReadMetrics);
                 loggedSql = sql;
+                logRequestReadMetricsFuncCallCount++;
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2edb87/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
index 53f5cb4..4ecde32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixResultSet.java
@@ -26,19 +26,26 @@ public class LoggingPhoenixResultSet extends DelegateResultSet {
     
     private PhoenixMetricsLog phoenixMetricsLog;
     private String sql;
+    private boolean areMetricsLogged;
 
     public LoggingPhoenixResultSet(ResultSet rs, PhoenixMetricsLog phoenixMetricsLog, String sql) {
         super(rs);
         this.phoenixMetricsLog = phoenixMetricsLog;
         this.sql = sql;
+        this.areMetricsLogged = false;
     }
     
     @Override
     public void close() throws SQLException {
-        phoenixMetricsLog.logOverAllReadRequestMetrics(PhoenixRuntime.getOverAllReadRequestMetricInfo(rs), sql);
-        phoenixMetricsLog.logRequestReadMetrics(PhoenixRuntime.getRequestReadMetricInfo(rs), sql);
-        PhoenixRuntime.resetMetrics(rs);
-        super.close();
+        if (!rs.isClosed()) {
+            super.close();
+        }
+        if (!this.areMetricsLogged) {
+            phoenixMetricsLog.logOverAllReadRequestMetrics(PhoenixRuntime.getOverAllReadRequestMetricInfo(rs), sql);
+            phoenixMetricsLog.logRequestReadMetrics(PhoenixRuntime.getRequestReadMetricInfo(rs), sql);
+            PhoenixRuntime.resetMetrics(rs);
+            this.areMetricsLogged = true;
+        }
     }
 
 }


[42/50] [abbrv] phoenix git commit: PHOENIX-4894 Missing Apache license in two QueryServer classes

Posted by ja...@apache.org.
PHOENIX-4894 Missing Apache license in two QueryServer classes


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

Branch: refs/heads/omid2
Commit: 1145eda5fed4fa0f89c460167d84538786710088
Parents: d105fbb
Author: Karan Mehta <ka...@gmail.com>
Authored: Fri Sep 7 10:30:57 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Fri Sep 7 10:50:02 2018 -0700

----------------------------------------------------------------------
 .../server/AvaticaServerConfigurationFactory.java  | 17 +++++++++++++++++
 .../CustomAvaticaServerConfigurationTest.java      | 17 +++++++++++++++++
 2 files changed, 34 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1145eda5/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
index 87a72ea..33fd590 100644
--- a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/AvaticaServerConfigurationFactory.java
@@ -1,3 +1,20 @@
+/*
+ * 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.queryserver.server;
 
 import org.apache.calcite.avatica.server.AvaticaServerConfiguration;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1145eda5/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
index 20bc868..fb59e0d 100644
--- a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/CustomAvaticaServerConfigurationTest.java
@@ -1,3 +1,20 @@
+/*
+ * 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.queryserver.server;
 
 import org.apache.calcite.avatica.server.AvaticaServerConfiguration;


[43/50] [abbrv] phoenix git commit: PHOENIX-4666 Persistent subquery cache for hash joins

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/012bb318/phoenix-protocol/src/main/build-proto.sh
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/build-proto.sh b/phoenix-protocol/src/main/build-proto.sh
index b80bf1d..555651b 100755
--- a/phoenix-protocol/src/main/build-proto.sh
+++ b/phoenix-protocol/src/main/build-proto.sh
@@ -27,6 +27,12 @@ if [ $? != 0 ] ; then
   exit 1
 fi
 
+if [[ `protoc --version` != *"2.5.0"* ]]; then
+    echo "Must use protoc version 2.5.0"
+    exit 1
+fi
+
+
 PROTO_ROOT_DIR=`dirname $0`
 PROTO_DIR=$PROTO_ROOT_DIR
 JAVA_DIR=$PROTO_ROOT_DIR/../../../phoenix-core/src/main/java


[25/50] [abbrv] phoenix git commit: PHOENIX-4843 InListExpression toString() converts the values in the list to ASC sort order always

Posted by ja...@apache.org.
PHOENIX-4843 InListExpression toString() converts the values in the list to ASC sort order always


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

Branch: refs/heads/omid2
Commit: a2a30443b72a0078e9b31bc9a36aee0be72d6578
Parents: 59f8d0f
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Aug 9 17:33:09 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Mon Aug 13 10:41:21 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/InListIT.java    | 55 ++++++++++++++++++++
 .../phoenix/expression/InListExpression.java    |  5 +-
 2 files changed, 58 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2a30443/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
index fe88dc8..871f326 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static java.util.Collections.singletonList;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -27,6 +28,7 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -46,6 +48,8 @@ import com.google.common.collect.Lists;
 
 
 public class InListIT extends ParallelStatsDisabledIT {
+    
+    private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
 
     @Test
     public void testLeadingPKWithTrailingRVC() throws Exception {
@@ -481,5 +485,56 @@ public class InListIT extends ParallelStatsDisabledIT {
         
         conn.close();
     }
+    
+    @Test
+    public void testInListExpressionWithDesc() throws Exception {
+        String fullTableName = generateUniqueName();
+        String fullViewName = generateUniqueName();
+        String tenantView = generateUniqueName();
+        // create base table and global view using global connection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            Statement stmt = conn.createStatement();
+            stmt.execute("CREATE TABLE " + fullTableName + "(\n" + 
+                    "    TENANT_ID CHAR(15) NOT NULL,\n" + 
+                    "    KEY_PREFIX CHAR(3) NOT NULL,\n" + 
+                    "    CREATED_DATE DATE,\n" + 
+                    "    CREATED_BY CHAR(15),\n" + 
+                    "    SYSTEM_MODSTAMP DATE\n" + 
+                    "    CONSTRAINT PK PRIMARY KEY (\n" + 
+                    "       TENANT_ID," + 
+                    "       KEY_PREFIX" + 
+                    ")) MULTI_TENANT=TRUE");
+            
+            stmt.execute("CREATE VIEW " + fullViewName + "(\n" + 
+                    "    MODEL VARCHAR NOT NULL,\n" + 
+                    "    MILEAGE  BIGINT NOT NULL,\n" +  
+                    "    MILES_DRIVEN BIGINT NOT NULL,\n" + 
+                    "    MAKE VARCHAR,\n" + 
+                    "    CONSTRAINT PKVIEW PRIMARY KEY\n" + 
+                    "    (\n" + 
+                    "    MODEL, MILEAGE DESC, MILES_DRIVEN\n" + 
+                    ")) AS SELECT * FROM " + fullTableName + " WHERE KEY_PREFIX = '0CY'");
+            
+        }
+        
+        // create and use a tenant specific view to write data
+        try (Connection viewConn = DriverManager.getConnection(TENANT_SPECIFIC_URL1) ) { 
+            Statement stmt = viewConn.createStatement();
+            stmt.execute("CREATE VIEW IF NOT EXISTS " + tenantView + " AS SELECT * FROM " + fullViewName );
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView + "(CREATED_BY, CREATED_DATE, SYSTEM_MODSTAMP, MODEL, MILEAGE, MILES_DRIVEN, MAKE) VALUES ('005xx000001Sv6o', 1532458254819, 1532458254819, 'a5', 23, 10000, 'AUDI')");
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView + "(CREATED_BY, CREATED_DATE, SYSTEM_MODSTAMP, MODEL, MILEAGE, MILES_DRIVEN, MAKE) VALUES ('005xx000001Sv6o', 1532458254819, 1532458254819, 'a4', 27, 30000, 'AUDI')");
+            viewConn.createStatement().execute("UPSERT INTO " + tenantView + "(CREATED_BY, CREATED_DATE, SYSTEM_MODSTAMP, MODEL, MILEAGE, MILES_DRIVEN, MAKE) VALUES ('005xx000001Sv6o', 1532458254819, 1532458254819, '328i', 32, 40000, 'BMW')");
+            viewConn.commit();
+            
+            ResultSet rs = stmt.executeQuery("SELECT Make, Model FROM " + tenantView + " WHERE MILEAGE IN (32, 27)");
+            assertTrue(rs.next());
+            assertEquals("BMW", rs.getString(1));
+            assertEquals("328i", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("AUDI", rs.getString(1));
+            assertEquals("a4", rs.getString(2));
+            assertFalse(rs.next());
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a2a30443/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
index a977f1f..a0a2ccc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
@@ -273,10 +273,11 @@ public class InListExpression extends BaseSingleExpression {
         PDataType type = firstChild.getDataType();
         StringBuilder buf = new StringBuilder(firstChild + " IN (");
         for (ImmutableBytesPtr value : values) {
+            ImmutableBytesPtr valueCopy = new ImmutableBytesPtr(value.copyBytes());
             if (firstChild.getSortOrder() != null) {
-                type.coerceBytes(value, type, firstChild.getSortOrder(), SortOrder.getDefault());
+                type.coerceBytes(valueCopy, type, firstChild.getSortOrder(), SortOrder.getDefault());
             }
-            buf.append(type.toStringLiteral(value, null));
+            buf.append(type.toStringLiteral(valueCopy, null));
             buf.append(',');
             if (buf.length() >= maxToStringLen) {
                 buf.append("... ");


[38/50] [abbrv] phoenix git commit: PHOENIX-3547 Supporting more number of indices per table.

Posted by ja...@apache.org.
PHOENIX-3547 Supporting more number of indices per table.


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

Branch: refs/heads/omid2
Commit: bef8d7dfc5e9143443932aabf1ecd8176afa8ff1
Parents: a3e0810
Author: Mahdi Salarkia <ms...@box.com>
Authored: Fri Aug 31 11:06:32 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Fri Aug 31 15:54:43 2018 -0700

----------------------------------------------------------------------
 .../end2end/BaseTenantSpecificViewIndexIT.java  |   4 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |   8 +-
 .../end2end/TenantSpecificViewIndexIT.java      |   6 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   |   2 +-
 .../java/org/apache/phoenix/end2end/ViewIT.java |  10 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |   4 +-
 .../index/GlobalIndexOptimizationIT.java        |   2 +-
 .../phoenix/end2end/index/IndexUsageIT.java     |   4 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |   2 +-
 .../end2end/index/MutableIndexFailureIT.java    |   4 +-
 .../phoenix/end2end/index/ViewIndexIT.java      |   6 +-
 .../IndexHalfStoreFileReaderGenerator.java      |   3 +-
 .../phoenix/compile/CreateTableCompiler.java    |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   4 +-
 .../apache/phoenix/compile/FromCompiler.java    |   2 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   2 +-
 .../compile/TupleProjectionCompiler.java        |   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   2 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   4 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |   2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  53 ++-
 .../phoenix/coprocessor/MetaDataProtocol.java   |  53 ++-
 .../coprocessor/generated/MetaDataProtos.java   | 471 +++++++++++++------
 .../coprocessor/generated/PTableProtos.java     | 156 ++++--
 .../generated/ServerCachingProtos.java          | 128 ++++-
 .../apache/phoenix/index/IndexMaintainer.java   |  15 +-
 .../index/PhoenixIndexFailurePolicy.java        |   4 +-
 .../apache/phoenix/iterate/ExplainTable.java    |  13 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   2 +
 .../query/ConnectionQueryServicesImpl.java      |   8 +
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../apache/phoenix/query/QueryConstants.java    |   5 +-
 .../apache/phoenix/schema/DelegateTable.java    |   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  21 +-
 .../java/org/apache/phoenix/schema/PTable.java  |   3 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  50 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |   4 +
 .../TenantSpecificViewIndexCompileTest.java     |   6 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   2 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   2 +-
 phoenix-protocol/src/main/MetaDataService.proto |   6 +-
 phoenix-protocol/src/main/PTable.proto          |   3 +-
 .../src/main/ServerCachingService.proto         |   1 +
 43 files changed, 782 insertions(+), 314 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
index 9bd689e..26e2860 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
@@ -148,9 +148,9 @@ public class BaseTenantSpecificViewIndexIT extends SplitSystemCatalogIT {
                             + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         } else {
             String expected = saltBuckets == null ? 
-                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName + " [-32768,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName + " [-9223372036854775808,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY" :
-                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _IDX_" + tableName + " [0,-32768,'" + tenantId + "','" + valuePrefix + "v2-1'] - ["+(saltBuckets.intValue()-1)+",-32768,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _IDX_" + tableName + " [0,-9223372036854775808,'" + tenantId + "','" + valuePrefix + "v2-1'] - ["+(saltBuckets.intValue()-1)+",-9223372036854775808,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
 
                   + "    SERVER FILTER BY FIRST KEY ONLY\n"
                   + "CLIENT MERGE SORT";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index 5728a20..c3d553a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -196,8 +196,8 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
                 queryPlan);
         } else {
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + Short.MIN_VALUE + ",51]"
-                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + Short.MIN_VALUE + ",51] - ["+(saltBuckets.intValue()-1)+"," + Short.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + Long.MIN_VALUE + ",51]"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + Long.MIN_VALUE + ",51] - ["+(saltBuckets.intValue()-1)+"," + Long.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
                             queryPlan);
         }
 
@@ -237,9 +237,9 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
         } else {
             physicalTableName = viewIndexPhysicalName;
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + (Short.MIN_VALUE+1) + ",'foo']\n"
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName +" [" + (Long.MIN_VALUE+1) + ",'foo']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY"
-                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + (Short.MIN_VALUE+1) + ",'foo'] - ["+(saltBuckets.intValue()-1)+"," + (Short.MIN_VALUE+1) + ",'foo']\n"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0," + (Long.MIN_VALUE+1) + ",'foo'] - ["+(saltBuckets.intValue()-1)+"," + (Long.MIN_VALUE+1) + ",'foo']\n"
                                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                                     + "CLIENT MERGE SORT",
                             QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index 31f3569..a317693 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -130,8 +130,8 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         String sequenceNameA = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName(tenantId2), isNamespaceEnabled);
         String sequenceNameB = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName(tenantId1), isNamespaceEnabled);
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceEnabled);
-        verifySequenceValue(isNamespaceEnabled? tenantId2 : null, sequenceNameA, sequenceSchemaName, -32767);
-        verifySequenceValue(isNamespaceEnabled? tenantId1 : null, sequenceNameB, sequenceSchemaName, -32767);
+        verifySequenceValue(isNamespaceEnabled? tenantId2 : null, sequenceNameA, sequenceSchemaName, -9223372036854775807L);
+        verifySequenceValue(isNamespaceEnabled? tenantId1 : null, sequenceNameB, sequenceSchemaName, -9223372036854775807L);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId2);
@@ -207,7 +207,7 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
                     + Bytes.toString(MetaDataUtil.getViewIndexPhysicalName(
                         SchemaUtil.getPhysicalTableName(Bytes.toBytes(tableName), isNamespaceMapped).toBytes()))
-                    + " [-32768,'" + tenantId + "','f']\n" + "    SERVER FILTER BY FIRST KEY ONLY",
+                    + " [-9223372036854775808,'" + tenantId + "','f']\n" + "    SERVER FILTER BY FIRST KEY ONLY",
                     QueryUtil.getExplainPlan(rs));
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 c2cf9e6..a27d8dc 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
@@ -188,7 +188,7 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
                             + " IS NULL AND " + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + " = '" + newSchemaName
                             + "' AND " + PhoenixDatabaseMetaData.SEQUENCE_NAME + "='" + newSequenceName + "'");
             assertTrue(rs.next());
-            assertEquals("-32765", rs.getString(1));
+            assertEquals("-9223372036854775805", rs.getString(1));
             rs = phxConn.createStatement().executeQuery("SELECT " + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
                     + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + "," + PhoenixDatabaseMetaData.CURRENT_VALUE + "  FROM "
                     + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE + " WHERE " + PhoenixDatabaseMetaData.TENANT_ID

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index c1a7ff5..5e465b4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -1258,9 +1258,9 @@ public class ViewIT extends SplitSystemCatalogIT {
                     queryPlan);
         } else {
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [" + Short.MIN_VALUE + ",51]"
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [" + Long.MIN_VALUE + ",51]"
                     : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " [0,"
-                            + Short.MIN_VALUE + ",51] - [" + (saltBuckets.intValue() - 1) + "," + Short.MIN_VALUE
+                            + Long.MIN_VALUE + ",51] - [" + (saltBuckets.intValue() - 1) + "," + Long.MIN_VALUE
                             + ",51]\nCLIENT MERGE SORT",
                     queryPlan);
         }
@@ -1302,10 +1302,10 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertEquals(
                     saltBuckets == null
                             ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName + " ["
-                                    + (Short.MIN_VALUE + 1) + ",'foo']\n" + "    SERVER FILTER BY FIRST KEY ONLY"
+                                    + (Long.MIN_VALUE + 1) + ",'foo']\n" + "    SERVER FILTER BY FIRST KEY ONLY"
                             : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER " + viewIndexPhysicalName
-                                    + " [0," + (Short.MIN_VALUE + 1) + ",'foo'] - [" + (saltBuckets.intValue() - 1)
-                                    + "," + (Short.MIN_VALUE + 1) + ",'foo']\n"
+                                    + " [0," + (Long.MIN_VALUE + 1) + ",'foo'] - [" + (saltBuckets.intValue() - 1)
+                                    + "," + (Long.MIN_VALUE + 1) + ",'foo']\n"
                                     + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
                     QueryUtil.getExplainPlan(rs));
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
index f3ee2cb..54528a9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
@@ -161,7 +161,7 @@ public class ChildViewsUseParentViewIndexIT extends ParallelStatsDisabledIT {
         ResultSet rs = conn.prepareStatement("EXPLAIN " + sql).executeQuery();
         String childViewScanKey = isChildView ? ",'Y'" : "";
         assertEquals(
-            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 3 KEYS OVER _IDX_" + baseTableName + " [-32768,'1'" + childViewScanKey + "] - [-32768,'3'" + childViewScanKey + "]\n" +
+            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 3 KEYS OVER _IDX_" + baseTableName + " [-9223372036854775808,'1'" + childViewScanKey + "] - [-9223372036854775808,'3'" + childViewScanKey + "]\n" +
             "    SERVER FILTER BY FIRST KEY ONLY",
             QueryUtil.getExplainPlan(rs));
         
@@ -264,7 +264,7 @@ public class ChildViewsUseParentViewIndexIT extends ParallelStatsDisabledIT {
                 " ORDER BY WO_ID, A_DATE DESC";
         ResultSet rs = conn.prepareStatement("EXPLAIN " + sql).executeQuery();
         assertEquals(
-            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 5 RANGES OVER _IDX_" + baseTableName + " [-32768,'00Dxxxxxxxxxxx1','003xxxxxxxxxxx1',*] - [-32768,'00Dxxxxxxxxxxx1','003xxxxxxxxxxx5',~'2016-01-01 06:00:00.000']\n" + 
+            "CLIENT PARALLEL 1-WAY SKIP SCAN ON 5 RANGES OVER _IDX_" + baseTableName + " [-9223372036854775808,'00Dxxxxxxxxxxx1','003xxxxxxxxxxx1',*] - [-9223372036854775808,'00Dxxxxxxxxxxx1','003xxxxxxxxxxx5',~'2016-01-01 06:00:00.000']\n" +
             "    SERVER FILTER BY FIRST KEY ONLY",
             QueryUtil.getExplainPlan(rs));
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
index 51e9de9..09003b0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
@@ -344,7 +344,7 @@ public class GlobalIndexOptimizationIT extends ParallelStatsDisabledIT {
                     "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "    SERVER FILTER BY V1 = 'a'\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_" + dataTableName + " \\[-32768,1\\] - \\[-32768,2\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_" + dataTableName + " \\[-9223372036854775808,1\\] - \\[-9223372036854775808,2\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY AND \"K2\" IN \\(3,4\\)\n" +
                     "    DYNAMIC SERVER FILTER BY \\(\"" + viewName + ".T_ID\", \"" + viewName + ".K1\", \"" + viewName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             assertTrue("Expected:\n" + expected + "\ndid not match\n" + actual, Pattern.matches(expected,actual));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
index 14b569a..f114010 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexUsageIT.java
@@ -496,7 +496,7 @@ public class IndexUsageIT extends ParallelStatsDisabledIT {
 	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [1,173]\n" + "CLIENT MERGE SORT",
 	                    queryPlan);
 	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + Short.MIN_VALUE + ",173]", queryPlan);
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + Long.MIN_VALUE + ",173]", queryPlan);
 	        }
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
@@ -516,7 +516,7 @@ public class IndexUsageIT extends ParallelStatsDisabledIT {
 	                    + ",'foo2_bar2']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
 	                    QueryUtil.getExplainPlan(rs));
 	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + (Short.MIN_VALUE + 1) + ",'foo2_bar2']\n"
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + (Long.MIN_VALUE + 1) + ",'foo2_bar2']\n"
 	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
 	        }
 	        rs = conn.createStatement().executeQuery(query);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 0dcf1d5..913d4b1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -199,7 +199,7 @@ public class LocalIndexIT extends BaseLocalIndexIT {
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
-        verifySequenceValue(null, sequenceName, sequenceSchemaName,-32767);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName,-9223372036854775807L);
         conn2.createStatement().executeQuery("SELECT * FROM " + tableName).next();
         conn1.createStatement().execute("DROP TABLE "+ tableName);
         verifySequenceNotExists(null, sequenceName, sequenceSchemaName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 8f88513..0611478 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -591,9 +591,9 @@ public class MutableIndexFailureIT extends BaseTest {
                         if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             int regionStartKeyLen = c.getEnvironment().getRegionInfo().getStartKey().length;
                             Cell firstCell = entry.getValue().get(0);
-                            short indexId = MetaDataUtil.getViewIndexIdDataType().getCodec().decodeShort(firstCell.getRowArray(), firstCell.getRowOffset() + regionStartKeyLen, SortOrder.getDefault());
+                            long indexId = MetaDataUtil.getViewIndexIdDataType().getCodec().decodeLong(firstCell.getRowArray(), firstCell.getRowOffset() + regionStartKeyLen, SortOrder.getDefault());
                             // Only throw for first local index as the test may have multiple local indexes
-                            if (indexId == Short.MIN_VALUE) {
+                            if (indexId == Long.MIN_VALUE) {
                                 throwException = true;
                                 break;
                             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 3ce6ae5..11cfc5c9 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
@@ -124,10 +124,10 @@ public class ViewIndexIT extends SplitSystemCatalogIT {
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(fullTableName), isNamespaceMapped);
         String seqName = getViewIndexSequenceName(PNameFactory.newName(fullTableName), null, !isNamespaceMapped);
         String seqSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(fullTableName), !isNamespaceMapped);
-        verifySequenceValue(null, sequenceName, sequenceSchemaName, -32767);
-        verifySequenceValue(null, sequenceName, sequenceSchemaName, -32767);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, -9223372036854775807L);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, -9223372036854775807L);
         conn1.createStatement().execute("CREATE INDEX " + indexName + "_2 ON " + fullViewName + " (v1)");
-        verifySequenceValue(null, sequenceName, sequenceSchemaName, -32766);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, -9223372036854775806L);
         // Check other format of sequence is not there as Sequences format is different for views/indexes created on
         // table which are namespace mapped and which are not.
         verifySequenceNotExists(null, seqName, seqSchemaName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index ab65456..02ee0e0 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -157,8 +157,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
                 for (PTable index : indexes) {
                     if (index.getIndexType() == IndexType.LOCAL) {
                         IndexMaintainer indexMaintainer = index.getIndexMaintainer(dataTable, conn);
-                        indexMaintainers.put(new ImmutableBytesWritable(MetaDataUtil
-                                .getViewIndexIdDataType().toBytes(index.getViewIndexId())),
+                        indexMaintainers.put(new ImmutableBytesWritable(index.getViewIndexType().toBytes(index.getViewIndexId())),
                             indexMaintainer);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index d67899c..c9fd3b2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -64,6 +64,7 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.QueryUtil;
 
 import com.google.common.collect.Iterators;
@@ -189,7 +190,7 @@ public class CreateTableCompiler {
             @Override
             public MutationState execute() throws SQLException {
                 try {
-                    return client.createTable(finalCreate, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced);
+                    return client.createTable(finalCreate, splits, parent, viewStatement, viewType, MetaDataUtil.getViewIndexIdDataType(), viewColumnConstants, isViewColumnReferenced);
                 } finally {
                     if (client.getConnection() != connection) {
                         client.getConnection().close();
@@ -340,4 +341,4 @@ public class CreateTableCompiler {
         }
         
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 5f9c76c..c2dfeab 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
@@ -156,7 +156,7 @@ public class DeleteCompiler {
         int offset = (table.getBucketNum() == null ? 0 : 1);
         byte[][] values = new byte[pkColumns.size()][];
         if (isSharedViewIndex) {
-            values[offset++] = MetaDataUtil.getViewIndexIdDataType().toBytes(table.getViewIndexId());
+            values[offset++] = table.getViewIndexType().toBytes(table.getViewIndexId());
         }
         if (isMultiTenant) {
             values[offset++] = tenantIdBytes;
@@ -977,4 +977,4 @@ public class DeleteCompiler {
         return table.getIndexType() != IndexType.LOCAL && (table.isImmutableRows() || table.isTransactional());
     }
     
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 4adc805..6eb5952 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
@@ -828,7 +828,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, null, 0, 0L, SchemaUtil
+                    false, null, 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/bef8d7df/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 824d933..36bfc5f 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
@@ -1275,7 +1275,7 @@ public class JoinCompiler {
                 left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(),
                 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.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexType(), left.getViewIndexId(),
                 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/bef8d7df/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 91be356..a2937bf 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
@@ -166,7 +166,7 @@ public class TupleProjectionCompiler {
                 table.getPKName(), table.getBucketNum(), projectedColumns, table.getParentSchemaName(),
                 table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName> emptyList(),
                 table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
-                table.getViewIndexId(),
+                table.getViewIndexType(), table.getViewIndexId(),
                 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.getTransactionProvider(),
+                table.getViewIndexType(), 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/bef8d7df/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 c94634b..c0c63eb 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
@@ -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, null, 0, 0L,
+            true, true, null, 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/bef8d7df/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 d0dd2cf..6d81f8b 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
@@ -726,7 +726,7 @@ public class UpsertCompiler {
         final byte[][] values = new byte[nValuesToSet][];
         int nodeIndex = 0;
         if (isSharedViewIndex) {
-            values[nodeIndex++] = MetaDataUtil.getViewIndexIdDataType().toBytes(table.getViewIndexId());
+            values[nodeIndex++] = table.getViewIndexType().toBytes(table.getViewIndexId());
         }
         if (isTenantSpecific) {
             PName tenantId = connection.getTenantId();
@@ -1347,4 +1347,4 @@ public class UpsertCompiler {
             return queryPlan.getEstimateInfoTimestamp();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index a5287cb..12e09d2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -179,7 +179,7 @@ public class WhereOptimizer {
         // Add unique index ID for shared indexes on views. This ensures
         // that different indexes don't interleave.
         if (hasViewIndex) {
-            byte[] viewIndexBytes = MetaDataUtil.getViewIndexIdDataType().toBytes(table.getViewIndexId());
+            byte[] viewIndexBytes = table.getViewIndexType().toBytes(table.getViewIndexId());
             KeyRange indexIdKeyRange = KeyRange.getKeyRange(viewIndexBytes);
             cnf.add(Collections.singletonList(indexIdKeyRange));
             pkPos++;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 e748115..cf5828d 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
@@ -74,6 +74,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT_BYTE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA_TYPE_BYTES;
 import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.schema.PTableType.TABLE;
@@ -316,6 +317,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue MULTI_TENANT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, MULTI_TENANT_BYTES);
     private static final KeyValue VIEW_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_TYPE_BYTES);
     private static final KeyValue VIEW_INDEX_ID_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_INDEX_ID_BYTES);
+    /**
+     * A designator for choosing the right type for viewIndex (Short vs Long) to be backward compatible.
+     * **/
+    private static final KeyValue VIEW_INDEX_ID_DATA_TYPE_BYTES_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_INDEX_ID_DATA_TYPE_BYTES);
     private static final KeyValue INDEX_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_TYPE_BYTES);
     private static final KeyValue INDEX_DISABLE_TIMESTAMP_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES);
     private static final KeyValue STORE_NULLS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, STORE_NULLS_BYTES);
@@ -349,6 +354,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             MULTI_TENANT_KV,
             VIEW_TYPE_KV,
             VIEW_INDEX_ID_KV,
+            VIEW_INDEX_ID_DATA_TYPE_BYTES_KV,
             INDEX_TYPE_KV,
             INDEX_DISABLE_TIMESTAMP_KV,
             STORE_NULLS_KV,
@@ -381,6 +387,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int DISABLE_WAL_INDEX = TABLE_KV_COLUMNS.indexOf(DISABLE_WAL_KV);
     private static final int MULTI_TENANT_INDEX = TABLE_KV_COLUMNS.indexOf(MULTI_TENANT_KV);
     private static final int VIEW_TYPE_INDEX = TABLE_KV_COLUMNS.indexOf(VIEW_TYPE_KV);
+    private static final int VIEW_INDEX_ID_DATA_TYPE_INDEX = TABLE_KV_COLUMNS.indexOf(VIEW_INDEX_ID_DATA_TYPE_BYTES_KV);
     private static final int VIEW_INDEX_ID_INDEX = TABLE_KV_COLUMNS.indexOf(VIEW_INDEX_ID_KV);
     private static final int INDEX_TYPE_INDEX = TABLE_KV_COLUMNS.indexOf(INDEX_TYPE_KV);
     private static final int STORE_NULLS_INDEX = TABLE_KV_COLUMNS.indexOf(STORE_NULLS_KV);
@@ -1345,8 +1352,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         Cell viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
         ViewType viewType = viewTypeKv == null ? null : ViewType.fromSerializedValue(viewTypeKv.getValueArray()[viewTypeKv.getValueOffset()]);
-        Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
-        Short viewIndexId = viewIndexIdKv == null ? null : (Short)MetaDataUtil.getViewIndexIdDataType().getCodec().decodeShort(viewIndexIdKv.getValueArray(), viewIndexIdKv.getValueOffset(), SortOrder.getDefault());
+        PDataType viewIndexType = getViewIndexType(tableKeyValues);
+        Long viewIndexId = getViewIndexId(tableKeyValues, viewIndexType);
         Cell indexTypeKv = tableKeyValues[INDEX_TYPE_INDEX];
         IndexType indexType = indexTypeKv == null ? null : IndexType.fromSerializedValue(indexTypeKv.getValueArray()[indexTypeKv.getValueOffset()]);
         Cell baseColumnCountKv = tableKeyValues[BASE_COLUMN_COUNT_INDEX];
@@ -1427,10 +1434,35 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // server while holding this lock is a bad idea and likely to cause contention.
         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,
+                viewStatement, disableWAL, multiTenant, storeNulls, viewType, viewIndexType, viewIndexId, indexType,
                 rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, baseColumnCount,
                 indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, encodingScheme, cqCounter, useStatsForParallelization);
     }
+    private Long getViewIndexId(Cell[] tableKeyValues, PDataType viewIndexType) {
+        Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
+        return viewIndexIdKv == null ? null :
+                decodeViewIndexId(viewIndexIdKv, viewIndexType);
+    }
+
+    /**
+     * Returns viewIndexId based on its underlying data type
+     *
+     * @param tableKeyValues
+     * @param viewIndexType
+     * @return
+     */
+    private Long decodeViewIndexId(Cell viewIndexIdKv, PDataType viewIndexType) {
+        return viewIndexType.getCodec().decodeLong(viewIndexIdKv.getValueArray(),
+                viewIndexIdKv.getValueOffset(), SortOrder.getDefault());
+    }
+
+    private PDataType getViewIndexType(Cell[] tableKeyValues) {
+        Cell dataTypeKv = tableKeyValues[VIEW_INDEX_ID_DATA_TYPE_INDEX];
+        return dataTypeKv == null ?
+                MetaDataUtil.getLegacyViewIndexIdDataType() :
+                PDataType.fromTypeId(PInteger.INSTANCE.getCodec()
+                        .decodeInt(dataTypeKv.getValueArray(), dataTypeKv.getValueOffset(), SortOrder.getDefault()));
+    }
 
     private boolean isQualifierCounterKV(Cell kv) {
         int cmp =
@@ -2147,7 +2179,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
                     cells.add(viewConstantCell);
                 }
-                Short indexId = null;
+                Long indexId = null;
                 if (request.hasAllocateIndexId() && request.getAllocateIndexId()) {
                     String tenantIdStr = tenantIdBytes.length == 0 ? null : Bytes.toString(tenantIdBytes);
                     try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) {
@@ -2161,7 +2193,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
                         try {
                             connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                                Short.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
+                                Long.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
                         } catch (SequenceAlreadyExistsException e) {
                         }
                         long[] seqValues = new long[1];
@@ -2172,12 +2204,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             throw sqlExceptions[0];
                         }
                         long seqValue = seqValues[0];
-                        if (seqValue > Short.MAX_VALUE) {
-                            builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
-                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                            done.run(builder.build());
-                            return;
-                        }
                         Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
 
                         NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
@@ -2190,7 +2216,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
                             cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
                         cells.add(indexIdCell);
-                        indexId = (short) seqValue;
+                        indexId = seqValue;
                     }
                 }
                 
@@ -2269,7 +2295,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 long currentTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                 if (indexId != null) {
-                    builder.setViewIndexId(indexId);
+                   builder.setViewIndexId(indexId);
+                   builder.setViewIndexType(PLong.INSTANCE.getSqlType());
                 }
                 builder.setMutationTime(currentTimeStamp);
                 done.run(builder.build());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 29cf2a3..115d45b 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
@@ -38,11 +38,13 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ByteUtil;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
+import org.apache.phoenix.util.MetaDataUtil;
 
 /**
  *
@@ -170,7 +172,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
         private PName tableName;
         private List<PColumn> columns;
         private List<PName> physicalNames;
-        private Short viewIndexId;
+        private PDataType viewIndexType;
+        private Long viewIndexId;
         
         public SharedTableState(PTable table) {
             this.tenantId = table.getTenantId();
@@ -178,6 +181,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
             this.tableName = table.getTableName();
             this.columns = table.getColumns();
             this.physicalNames = table.getPhysicalNames();
+            this.viewIndexType = table.getViewIndexType();
             this.viewIndexId = table.getViewIndexId();
         }
         
@@ -200,7 +204,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
                     return PNameFactory.newName(physicalName.toByteArray());
                 }
             });
-            this.viewIndexId = (short)sharedTable.getViewIndexId();
+            this.viewIndexId = sharedTable.getViewIndexId();
+            this.viewIndexType = sharedTable.hasViewIndexType()
+                    ? PDataType.fromTypeId(sharedTable.getViewIndexType())
+                    : MetaDataUtil.getLegacyViewIndexIdDataType();
         }
 
         public PName getTenantId() {
@@ -223,10 +230,13 @@ public abstract class MetaDataProtocol extends MetaDataService {
             return physicalNames;
         }
 
-        public Short getViewIndexId() {
+        public Long getViewIndexId() {
             return viewIndexId;
         }
-        
+
+        public PDataType getViewIndexType() {
+          return viewIndexType;
+        }
   }
     
   public static class MetaDataMutationResult {
@@ -239,8 +249,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
         private byte[] familyName;
         private boolean wasUpdated;
         private PSchema schema;
-        private Short viewIndexId;
-
+        private Long viewIndexId;
+        private PDataType viewIndexType;
         private List<PFunction> functions = new ArrayList<PFunction>(1);
         private long autoPartitionNum;
 
@@ -285,9 +295,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
             this.tableNamesToDelete = tableNamesToDelete;
         }
         
-        public MetaDataMutationResult(MutationCode returnCode, int currentTime, PTable table, int viewIndexId) {
+        public MetaDataMutationResult(MutationCode returnCode, int currentTime, PTable table, long viewIndexId, PDataType viewIndexType ) {
             this(returnCode, currentTime, table, Collections.<byte[]> emptyList());
-            this.viewIndexId = (short)viewIndexId;
+            this.viewIndexId = viewIndexId;
+            this.viewIndexType = viewIndexType;
         }
         
         public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, List<byte[]> tableNamesToDelete, List<SharedTableState> sharedTablesToDelete) {
@@ -343,10 +354,14 @@ public abstract class MetaDataProtocol extends MetaDataService {
             return autoPartitionNum;
         }
         
-        public Short getViewIndexId() {
+        public Long getViewIndexId() {
             return viewIndexId;
         }
 
+      public PDataType getViewIndexType() {
+          return viewIndexType;
+      }
+
         public static MetaDataMutationResult constructFromProto(MetaDataResponse proto) {
           MetaDataMutationResult result = new MetaDataMutationResult();
           result.returnCode = MutationCode.values()[proto.getReturnCode().ordinal()];
@@ -388,9 +403,13 @@ public abstract class MetaDataProtocol extends MetaDataService {
           if (proto.hasAutoPartitionNum()) {
               result.autoPartitionNum = proto.getAutoPartitionNum();
           }
-            if (proto.hasViewIndexId()) {
-                result.viewIndexId = (short)proto.getViewIndexId();
-            }
+          if (proto.hasViewIndexId()) {
+               result.viewIndexId = proto.getViewIndexId();
+          }
+
+          result.viewIndexType = proto.hasViewIndexType()
+                    ? PDataType.fromTypeId(proto.getViewIndexType())
+                    : MetaDataUtil.getLegacyViewIndexIdDataType();
           return result;
         }
 
@@ -431,6 +450,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
                 sharedTableStateBuilder.setSchemaName(ByteStringer.wrap(sharedTableState.getSchemaName().getBytes()));
                 sharedTableStateBuilder.setTableName(ByteStringer.wrap(sharedTableState.getTableName().getBytes()));
                 sharedTableStateBuilder.setViewIndexId(sharedTableState.getViewIndexId());
+                sharedTableStateBuilder.setViewIndexType(sharedTableState.viewIndexType.getSqlType());
                 builder.addSharedTablesToDelete(sharedTableStateBuilder.build());
               }
             }
@@ -438,9 +458,12 @@ public abstract class MetaDataProtocol extends MetaDataService {
               builder.setSchema(PSchema.toProto(result.schema));
             }
             builder.setAutoPartitionNum(result.getAutoPartitionNum());
-                if (result.getViewIndexId() != null) {
-                    builder.setViewIndexId(result.getViewIndexId());
-                }
+            if (result.getViewIndexId() != null) {
+                builder.setViewIndexId(result.getViewIndexId());
+            }
+            builder.setViewIndexType(result.getViewIndexType() == null
+					  ? MetaDataUtil.getLegacyViewIndexIdDataType().getSqlType()
+					  : result.getViewIndexType().getSqlType());
           }
           return builder.build();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bef8d7df/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 744dc7e..360dd77 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
@@ -369,15 +369,25 @@ public final class MetaDataProtos {
      */
     com.google.protobuf.ByteString getPhysicalNames(int index);
 
-    // required int32 viewIndexId = 6;
+    // required int64 viewIndexId = 6;
     /**
-     * <code>required int32 viewIndexId = 6;</code>
+     * <code>required int64 viewIndexId = 6;</code>
      */
     boolean hasViewIndexId();
     /**
-     * <code>required int32 viewIndexId = 6;</code>
+     * <code>required int64 viewIndexId = 6;</code>
      */
-    int getViewIndexId();
+    long getViewIndexId();
+
+    // optional int32 viewIndexType = 7 [default = 5];
+    /**
+     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     */
+    boolean hasViewIndexType();
+    /**
+     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     */
+    int getViewIndexType();
   }
   /**
    * Protobuf type {@code SharedTableState}
@@ -463,7 +473,12 @@ public final class MetaDataProtos {
             }
             case 48: {
               bitField0_ |= 0x00000008;
-              viewIndexId_ = input.readInt32();
+              viewIndexId_ = input.readInt64();
+              break;
+            }
+            case 56: {
+              bitField0_ |= 0x00000010;
+              viewIndexType_ = input.readInt32();
               break;
             }
           }
@@ -619,29 +634,46 @@ public final class MetaDataProtos {
       return physicalNames_.get(index);
     }
 
-    // required int32 viewIndexId = 6;
+    // required int64 viewIndexId = 6;
     public static final int VIEWINDEXID_FIELD_NUMBER = 6;
-    private int viewIndexId_;
+    private long viewIndexId_;
     /**
-     * <code>required int32 viewIndexId = 6;</code>
+     * <code>required int64 viewIndexId = 6;</code>
      */
     public boolean hasViewIndexId() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>required int32 viewIndexId = 6;</code>
+     * <code>required int64 viewIndexId = 6;</code>
      */
-    public int getViewIndexId() {
+    public long getViewIndexId() {
       return viewIndexId_;
     }
 
+    // optional int32 viewIndexType = 7 [default = 5];
+    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 7;
+    private int viewIndexType_;
+    /**
+     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     */
+    public boolean hasViewIndexType() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     */
+    public int getViewIndexType() {
+      return viewIndexType_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       schemaName_ = com.google.protobuf.ByteString.EMPTY;
       tableName_ = com.google.protobuf.ByteString.EMPTY;
       columns_ = java.util.Collections.emptyList();
       physicalNames_ = java.util.Collections.emptyList();
-      viewIndexId_ = 0;
+      viewIndexId_ = 0L;
+      viewIndexType_ = 5;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -689,7 +721,10 @@ public final class MetaDataProtos {
         output.writeBytes(5, physicalNames_.get(i));
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeInt32(6, viewIndexId_);
+        output.writeInt64(6, viewIndexId_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeInt32(7, viewIndexType_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -727,7 +762,11 @@ public final class MetaDataProtos {
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(6, viewIndexId_);
+          .computeInt64Size(6, viewIndexId_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(7, viewIndexType_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -776,6 +815,11 @@ public final class MetaDataProtos {
         result = result && (getViewIndexId()
             == other.getViewIndexId());
       }
+      result = result && (hasViewIndexType() == other.hasViewIndexType());
+      if (hasViewIndexType()) {
+        result = result && (getViewIndexType()
+            == other.getViewIndexType());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -811,7 +855,11 @@ public final class MetaDataProtos {
       }
       if (hasViewIndexId()) {
         hash = (37 * hash) + VIEWINDEXID_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexId();
+        hash = (53 * hash) + hashLong(getViewIndexId());
+      }
+      if (hasViewIndexType()) {
+        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexType();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -937,8 +985,10 @@ public final class MetaDataProtos {
         }
         physicalNames_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000010);
-        viewIndexId_ = 0;
+        viewIndexId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000020);
+        viewIndexType_ = 5;
+        bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
 
@@ -997,6 +1047,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000008;
         }
         result.viewIndexId_ = viewIndexId_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.viewIndexType_ = viewIndexType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1061,6 +1115,9 @@ public final class MetaDataProtos {
         if (other.hasViewIndexId()) {
           setViewIndexId(other.getViewIndexId());
         }
+        if (other.hasViewIndexType()) {
+          setViewIndexType(other.getViewIndexType());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1526,35 +1583,68 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // required int32 viewIndexId = 6;
-      private int viewIndexId_ ;
+      // required int64 viewIndexId = 6;
+      private long viewIndexId_ ;
       /**
-       * <code>required int32 viewIndexId = 6;</code>
+       * <code>required int64 viewIndexId = 6;</code>
        */
       public boolean hasViewIndexId() {
         return ((bitField0_ & 0x00000020) == 0x00000020);
       }
       /**
-       * <code>required int32 viewIndexId = 6;</code>
+       * <code>required int64 viewIndexId = 6;</code>
        */
-      public int getViewIndexId() {
+      public long getViewIndexId() {
         return viewIndexId_;
       }
       /**
-       * <code>required int32 viewIndexId = 6;</code>
+       * <code>required int64 viewIndexId = 6;</code>
        */
-      public Builder setViewIndexId(int value) {
+      public Builder setViewIndexId(long value) {
         bitField0_ |= 0x00000020;
         viewIndexId_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required int32 viewIndexId = 6;</code>
+       * <code>required int64 viewIndexId = 6;</code>
        */
       public Builder clearViewIndexId() {
         bitField0_ = (bitField0_ & ~0x00000020);
-        viewIndexId_ = 0;
+        viewIndexId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional int32 viewIndexType = 7 [default = 5];
+      private int viewIndexType_ = 5;
+      /**
+       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       */
+      public boolean hasViewIndexType() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       */
+      public int getViewIndexType() {
+        return viewIndexType_;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       */
+      public Builder setViewIndexType(int value) {
+        bitField0_ |= 0x00000040;
+        viewIndexType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       */
+      public Builder clearViewIndexType() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        viewIndexType_ = 5;
         onChanged();
         return this;
       }
@@ -1725,15 +1815,25 @@ public final class MetaDataProtos {
      */
     long getAutoPartitionNum();
 
-    // optional int32 viewIndexId = 12;
+    // optional int64 viewIndexId = 12;
     /**
-     * <code>optional int32 viewIndexId = 12;</code>
+     * <code>optional int64 viewIndexId = 12;</code>
      */
     boolean hasViewIndexId();
     /**
-     * <code>optional int32 viewIndexId = 12;</code>
+     * <code>optional int64 viewIndexId = 12;</code>
      */
-    int getViewIndexId();
+    long getViewIndexId();
+
+    // optional int32 viewIndexType = 13 [default = 5];
+    /**
+     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     */
+    boolean hasViewIndexType();
+    /**
+     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     */
+    int getViewIndexType();
   }
   /**
    * Protobuf type {@code MetaDataResponse}
@@ -1874,7 +1974,12 @@ public final class MetaDataProtos {
             }
             case 96: {
               bitField0_ |= 0x00000100;
-              viewIndexId_ = input.readInt32();
+              viewIndexId_ = input.readInt64();
+              break;
+            }
+            case 104: {
+              bitField0_ |= 0x00000200;
+              viewIndexType_ = input.readInt32();
               break;
             }
           }
@@ -2161,22 +2266,38 @@ public final class MetaDataProtos {
       return autoPartitionNum_;
     }
 
-    // optional int32 viewIndexId = 12;
+    // optional int64 viewIndexId = 12;
     public static final int VIEWINDEXID_FIELD_NUMBER = 12;
-    private int viewIndexId_;
+    private long viewIndexId_;
     /**
-     * <code>optional int32 viewIndexId = 12;</code>
+     * <code>optional int64 viewIndexId = 12;</code>
      */
     public boolean hasViewIndexId() {
       return ((bitField0_ & 0x00000100) == 0x00000100);
     }
     /**
-     * <code>optional int32 viewIndexId = 12;</code>
+     * <code>optional int64 viewIndexId = 12;</code>
      */
-    public int getViewIndexId() {
+    public long getViewIndexId() {
       return viewIndexId_;
     }
 
+    // optional int32 viewIndexType = 13 [default = 5];
+    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 13;
+    private int viewIndexType_;
+    /**
+     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     */
+    public boolean hasViewIndexType() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    /**
+     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     */
+    public int getViewIndexType() {
+      return viewIndexType_;
+    }
+
     private void initFields() {
       returnCode_ = org.apache.phoenix.coprocessor.generated.MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS;
       mutationTime_ = 0L;
@@ -2189,7 +2310,8 @@ public final class MetaDataProtos {
       sharedTablesToDelete_ = java.util.Collections.emptyList();
       schema_ = org.apache.phoenix.coprocessor.generated.PSchemaProtos.PSchema.getDefaultInstance();
       autoPartitionNum_ = 0L;
-      viewIndexId_ = 0;
+      viewIndexId_ = 0L;
+      viewIndexType_ = 5;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2261,7 +2383,10 @@ public final class MetaDataProtos {
         output.writeInt64(11, autoPartitionNum_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        output.writeInt32(12, viewIndexId_);
+        output.writeInt64(12, viewIndexId_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeInt32(13, viewIndexType_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -2323,7 +2448,11 @@ public final class MetaDataProtos {
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(12, viewIndexId_);
+          .computeInt64Size(12, viewIndexId_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(13, viewIndexType_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -2399,6 +2528,11 @@ public final class MetaDataProtos {
         result = result && (getViewIndexId()
             == other.getViewIndexId());
       }
+      result = result && (hasViewIndexType() == other.hasViewIndexType());
+      if (hasViewIndexType()) {
+        result = result && (getViewIndexType()
+            == other.getViewIndexType());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -2458,7 +2592,11 @@ public final class MetaDataProtos {
       }
       if (hasViewIndexId()) {
         hash = (37 * hash) + VIEWINDEXID_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexId();
+        hash = (53 * hash) + hashLong(getViewIndexId());
+      }
+      if (hasViewIndexType()) {
+        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexType();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -2611,8 +2749,10 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000200);
         autoPartitionNum_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000400);
-        viewIndexId_ = 0;
+        viewIndexId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000800);
+        viewIndexType_ = 5;
+        bitField0_ = (bitField0_ & ~0x00001000);
         return this;
       }
 
@@ -2708,6 +2848,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000100;
         }
         result.viewIndexId_ = viewIndexId_;
+        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
+          to_bitField0_ |= 0x00000200;
+        }
+        result.viewIndexType_ = viewIndexType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2813,6 +2957,9 @@ public final class MetaDataProtos {
         if (other.hasViewIndexId()) {
           setViewIndexId(other.getViewIndexId());
         }
+        if (other.hasViewIndexType()) {
+          setViewIndexType(other.getViewIndexType());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3860,35 +4007,68 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // optional int32 viewIndexId = 12;
-      private int viewIndexId_ ;
+      // optional int64 viewIndexId = 12;
+      private long viewIndexId_ ;
       /**
-       * <code>optional int32 viewIndexId = 12;</code>
+       * <code>optional int64 viewIndexId = 12;</code>
        */
       public boolean hasViewIndexId() {
         return ((bitField0_ & 0x00000800) == 0x00000800);
       }
       /**
-       * <code>optional int32 viewIndexId = 12;</code>
+       * <code>optional int64 viewIndexId = 12;</code>
        */
-      public int getViewIndexId() {
+      public long getViewIndexId() {
         return viewIndexId_;
       }
       /**
-       * <code>optional int32 viewIndexId = 12;</code>
+       * <code>optional int64 viewIndexId = 12;</code>
        */
-      public Builder setViewIndexId(int value) {
+      public Builder setViewIndexId(long value) {
         bitField0_ |= 0x00000800;
         viewIndexId_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 viewIndexId = 12;</code>
+       * <code>optional int64 viewIndexId = 12;</code>
        */
       public Builder clearViewIndexId() {
         bitField0_ = (bitField0_ & ~0x00000800);
-        viewIndexId_ = 0;
+        viewIndexId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional int32 viewIndexType = 13 [default = 5];
+      private int viewIndexType_ = 5;
+      /**
+       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       */
+      public boolean hasViewIndexType() {
+        return ((bitField0_ & 0x00001000) == 0x00001000);
+      }
+      /**
+       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       */
+      public int getViewIndexType() {
+        return viewIndexType_;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       */
+      public Builder setViewIndexType(int value) {
+        bitField0_ |= 0x00001000;
+        viewIndexType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       */
+      public Builder clearViewIndexType() {
+        bitField0_ = (bitField0_ & ~0x00001000);
+        viewIndexType_ = 5;
         onChanged();
         return this;
       }
@@ -17647,104 +17827,105 @@ public final class MetaDataProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\025MetaDataService.proto\032\014PTable.proto\032\017P" +
-      "Function.proto\032\rPSchema.proto\"\222\001\n\020Shared" +
+      "Function.proto\032\rPSchema.proto\"\254\001\n\020Shared" +
       "TableState\022\020\n\010tenantId\030\001 \001(\014\022\022\n\nschemaNa" +
       "me\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\031\n\007columns\030\004" +
       " \003(\0132\010.PColumn\022\025\n\rphysicalNames\030\005 \003(\014\022\023\n" +
-      "\013viewIndexId\030\006 \002(\005\"\321\002\n\020MetaDataResponse\022" +
-      "!\n\nreturnCode\030\001 \001(\0162\r.MutationCode\022\024\n\014mu" +
-      "tationTime\030\002 \001(\003\022\026\n\005table\030\003 \001(\0132\007.PTable" +
-      "\022\026\n\016tablesToDelete\030\004 \003(\014\022\022\n\ncolumnName\030\005" +
-      " \001(\014\022\022\n\nfamilyName\030\006 \001(\014\022\024\n\014functionName",
-      "\030\007 \001(\014\022\034\n\010function\030\010 \003(\0132\n.PFunction\022/\n\024" +
-      "sharedTablesToDelete\030\t \003(\0132\021.SharedTable" +
-      "State\022\030\n\006schema\030\n \001(\0132\010.PSchema\022\030\n\020autoP" +
-      "artitionNum\030\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\005\"\364" +
-      "\001\n\017GetTableRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\n" +
-      "schemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\026\n\016t" +
-      "ableTimestamp\030\004 \002(\003\022\027\n\017clientTimestamp\030\005" +
-      " \002(\003\022\025\n\rclientVersion\030\006 \001(\005\022\037\n\027skipAddin" +
-      "gParentColumns\030\007 \001(\010\022\031\n\021skipAddingIndexe" +
-      "s\030\010 \001(\010\022$\n\023lockedAncestorTable\030\t \001(\0132\007.P",
-      "Table\"\212\001\n\023GetFunctionsRequest\022\020\n\010tenantI" +
-      "d\030\001 \002(\014\022\025\n\rfunctionNames\030\002 \003(\014\022\032\n\022functi" +
-      "onTimestamps\030\003 \003(\003\022\027\n\017clientTimestamp\030\004 " +
-      "\002(\003\022\025\n\rclientVersion\030\005 \001(\005\"V\n\020GetSchemaR" +
-      "equest\022\022\n\nschemaName\030\001 \002(\t\022\027\n\017clientTime" +
-      "stamp\030\002 \002(\003\022\025\n\rclientVersion\030\003 \002(\005\"d\n\022Cr" +
-      "eateTableRequest\022\036\n\026tableMetadataMutatio" +
-      "ns\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\022\027\n\017alloc" +
-      "ateIndexId\030\003 \001(\010\"r\n\025CreateFunctionReques" +
-      "t\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\021\n\ttem",
-      "porary\030\002 \002(\010\022\017\n\007replace\030\003 \001(\010\022\025\n\rclientV" +
-      "ersion\030\004 \001(\005\"`\n\023CreateSchemaRequest\022\036\n\026t" +
-      "ableMetadataMutations\030\001 \003(\014\022\022\n\nschemaNam" +
-      "e\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"\216\001\n\020DropT" +
-      "ableRequest\022\036\n\026tableMetadataMutations\030\001 " +
-      "\003(\014\022\021\n\ttableType\030\002 \002(\t\022\017\n\007cascade\030\003 \001(\010\022" +
-      "\025\n\rclientVersion\030\004 \001(\005\022\037\n\027skipAddingPare" +
-      "ntColumns\030\005 \001(\010\"_\n\021DropSchemaRequest\022\037\n\027" +
-      "schemaMetadataMutations\030\001 \003(\014\022\022\n\nschemaN" +
-      "ame\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"I\n\020AddC",
-      "olumnRequest\022\036\n\026tableMetadataMutations\030\001" +
-      " \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"J\n\021DropColum" +
-      "nRequest\022\036\n\026tableMetadataMutations\030\001 \003(\014" +
-      "\022\025\n\rclientVersion\030\002 \001(\005\"^\n\023DropFunctionR" +
-      "equest\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\020" +
-      "\n\010ifExists\030\002 \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"" +
-      "P\n\027UpdateIndexStateRequest\022\036\n\026tableMetad" +
-      "ataMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(" +
-      "\005\"*\n\021ClearCacheRequest\022\025\n\rclientVersion\030" +
-      "\001 \001(\005\"*\n\022ClearCacheResponse\022\024\n\014unfreedBy",
-      "tes\030\001 \001(\003\"*\n\021GetVersionRequest\022\025\n\rclient" +
-      "Version\030\001 \001(\005\"E\n\022GetVersionResponse\022\017\n\007v" +
-      "ersion\030\001 \002(\003\022\036\n\026systemCatalogTimestamp\030\002" +
-      " \001(\003\"\205\001\n\032ClearTableFromCacheRequest\022\020\n\010t" +
-      "enantId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttab" +
-      "leName\030\003 \002(\014\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n" +
-      "\rclientVersion\030\005 \001(\005\"\035\n\033ClearTableFromCa" +
-      "cheResponse*\271\005\n\014MutationCode\022\030\n\024TABLE_AL" +
-      "READY_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020C" +
-      "OLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXIS",
-      "TS\020\003\022\035\n\031CONCURRENT_TABLE_MUTATION\020\004\022\027\n\023T" +
-      "ABLE_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABLE_FOUN" +
-      "D\020\006\022\034\n\030UNALLOWED_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\022FUNCTIO" +
-      "N_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_AL" +
-      "READY_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_RE" +
-      "GION\020\021\022\032\n\026TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UN",
-      "ALLOWED_SCHEMA_MUTATION\020\023\022%\n!AUTO_PARTIT" +
-      "ION_SEQUENCE_NOT_FOUND\020\024\022#\n\037CANNOT_COERC" +
-      "E_AUTO_PARTITION_ID\020\025\022\024\n\020TOO_MANY_INDEXE" +
-      "S\020\026\022\037\n\033UNABLE_TO_CREATE_CHILD_LINK\020\027\022!\n\035" +
-      "UNABLE_TO_UPDATE_PARENT_TABLE\020\0302\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"
+      "\013viewIndexId\030\006 \002(\003\022\030\n\rviewIndexType\030\007 \001(" +
+      "\005:\0015\"\353\002\n\020MetaDataResponse\022!\n\nreturnCode\030" +
+      "\001 \001(\0162\r.MutationCode\022\024\n\014mutationTime\030\002 \001" +
+      "(\003\022\026\n\005table\030\003 \001(\0132\007.PTable\022\026\n\016tablesToDe" +
+      "lete\030\004 \003(\014\022\022\n\ncolumnName\030\005 \001(\014\022\022\n\nfamily",
+      "Name\030\006 \001(\014\022\024\n\014functionName\030\007 \001(\014\022\034\n\010func" +
+      "tion\030\010 \003(\0132\n.PFunction\022/\n\024sharedTablesTo" +
+      "Delete\030\t \003(\0132\021.SharedTableState\022\030\n\006schem" +
+      "a\030\n \001(\0132\010.PSchema\022\030\n\020autoPartitionNum\030\013 " +
+      "\001(\003\022\023\n\013viewIndexId\030\014 \001(\003\022\030\n\rviewIndexTyp" +
+      "e\030\r \001(\005:\0015\"\364\001\n\017GetTableRequest\022\020\n\010tenant" +
+      "Id\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableNam" +
+      "e\030\003 \002(\014\022\026\n\016tableTimestamp\030\004 \002(\003\022\027\n\017clien" +
+      "tTimestamp\030\005 \002(\003\022\025\n\rclientVersion\030\006 \001(\005\022" +
+      "\037\n\027skipAddingParentColumns\030\007 \001(\010\022\031\n\021skip",
+      "AddingIndexes\030\010 \001(\010\022$\n\023lockedAncestorTab" +
+      "le\030\t \001(\0132\007.PTable\"\212\001\n\023GetFunctionsReques" +
+      "t\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctionNames\030\002 \003" +
+      "(\014\022\032\n\022functionTimestamps\030\003 \003(\003\022\027\n\017client" +
+      "Timestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"V" +
+      "\n\020GetSchemaRequest\022\022\n\nschemaName\030\001 \002(\t\022\027" +
+      "\n\017clientTimestamp\030\002 \002(\003\022\025\n\rclientVersion" +
+      "\030\003 \002(\005\"d\n\022CreateTableRequest\022\036\n\026tableMet" +
+      "adataMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 " +
+      "\001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r\n\025CreateFu",
+      "nctionRequest\022\036\n\026tableMetadataMutations\030" +
+      "\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007replace\030\003 \001(" +
+      "\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023CreateSchema" +
+      "Request\022\036\n\026tableMetadataMutations\030\001 \003(\014\022" +
+      "\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002" +
+      "(\005\"\216\001\n\020DropTableRequest\022\036\n\026tableMetadata" +
+      "Mutations\030\001 \003(\014\022\021\n\ttableType\030\002 \002(\t\022\017\n\007ca" +
+      "scade\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\022\037\n\027sk" +
+      "ipAddingParentColumns\030\005 \001(\010\"_\n\021DropSchem" +
+      "aRequest\022\037\n\027schemaMetadataMutations\030\001 \003(",
+      "\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion\030\003" +
+      " \002(\005\"I\n\020AddColumnRequest\022\036\n\026tableMetadat" +
+      "aMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"" +
+      "J\n\021DropColumnRequest\022\036\n\026tableMetadataMut" +
+      "ations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"^\n\023D" +
+      "ropFunctionRequest\022\036\n\026tableMetadataMutat" +
+      "ions\030\001 \003(\014\022\020\n\010ifExists\030\002 \001(\010\022\025\n\rclientVe" +
+      "rsion\030\003 \001(\005\"P\n\027UpdateIndexStateRequest\022\036" +
+      "\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclient" +
+      "Version\030\002 \001(\005\"*\n\021ClearCacheRequest\022\025\n\rcl",
+      "ientVersion\030\001 \001(\005\"*\n\022ClearCacheResponse\022" +
+      "\024\n\014unfreedBytes\030\001 \001(\003\"*\n\021GetVersionReque" +
+      "st\022\025\n\rclientVersion\030\001 \001(\005\"E\n\022GetVersionR" +
+      "esponse\022\017\n\007version\030\001 \002(\003\022\036\n\026systemCatalo" +
+      "gTimestamp\030\002 \001(\003\"\205\001\n\032ClearTableFromCache" +
+      "Request\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\017clientTimest" +
+      "amp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"\035\n\033Clea" +
+      "rTableFromCacheResponse*\271\005\n\014MutationCode" +
+      "\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_NOT_",
+      "FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_" +
+      "ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE_MUT" +
+      "ATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021NEWE" +
+      "R_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_MUTAT" +
+      "ION\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_FUNCT" +
+      "ION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_REGION\020\r\022" +
+      "\031\n\025SCHEMA_ALREADY_EXISTS\020\016\022\026\n\022NEWER_SCHE" +
+      "MA_FOUND\020\017\022\024\n\020SCHEMA_NOT_FOUND\020\020\022\030\n\024SCHE",
+      "MA_NOT_IN_REGION\020\021\022\032\n\026TABLES_EXIST_ON_SC" +
+      "HEMA\020\022\022\035\n\031UNALLOWED_SCHEMA_MUTATION\020\023\022%\n" +
+      "!AUTO_PARTITION_SEQUENCE_NOT_FOUND\020\024\022#\n\037" +
+      "CANNOT_COERCE_AUTO_PARTITION_ID\020\025\022\024\n\020TOO" +
+      "_MANY_INDEXES\020\026\022\037\n\033UNABLE_TO_CREATE_CHIL" +
+      "D_LINK\020\027\022!\n\035UNABLE_TO_UPDATE_PARENT_TABL" +
+      "E\020\0302\345\006\n\017MetaDataService\022/\n\010getTable\022\020.Ge" +
+      "tTableRequest\032\021.MetaDataResponse\0227\n\014getF" +
+      "unctions\022\024.GetFunctionsRequest\032\021.MetaDat" +
+      "aResponse\0221\n\tgetSchema\022\021.GetSchemaReques",
+      "t\032\021.MetaDataResponse\0225\n\013createTable\022\023.Cr" +
+      "eateTableRequest\032\021.MetaDataResponse\022;\n\016c" +
+      "reateFunction\022\026.CreateFunctionRequest\032\021." +
+      "MetaDataResponse\0227\n\014createSchema\022\024.Creat" +
+      "eSchemaRequest\032\021.MetaDataResponse\0221\n\tdro" +
+      "pTable\022\021.DropTableRequest\032\021.MetaDataResp" +
+      "onse\0223\n\ndropSchema\022\022.DropSchemaRequest\032\021" +
+      ".MetaDataResponse\0227\n\014dropFunction\022\024.Drop" +
+      "FunctionRequest\032\021.MetaDataResponse\0221\n\tad" +
+      "dColumn\022\021.AddColumnRequest\032\021.MetaDataRes",
+      "ponse\0223\n\ndropColumn\022\022.DropColumnRequest\032" +
+      "\021.MetaDataResponse\022?\n\020updateIndexState\022\030" +
+      ".UpdateIndexStateRequest\032\021.MetaDataRespo" +
+      "nse\0225\n\nclearCache\022\022.ClearCacheRequest\032\023." +
+      "ClearCacheResponse\0225\n\ngetVersion\022\022.GetVe" +
+      "rsionRequest\032\023.GetVersionResponse\022P\n\023cle" +
+      "arTableFromCache\022\033.ClearTableFromCacheRe" +
+      "quest\032\034.ClearTableFromCacheResponseBB\n(o" +
+      "rg.apache.phoenix.coprocessor.generatedB" +
+      "\016MetaDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17756,13 +17937,13 @@ public final class MetaDataProtos {
           internal_static_SharedTableState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_SharedTableState_descriptor,
-              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "Columns", "PhysicalNames", "ViewIndexId", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "Columns", "PhysicalNames", "ViewIndexId", "ViewIndexType", });
           internal_static_MetaDataResponse_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_MetaDataResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_MetaDataResponse_descriptor,
-              new java.lang.String[] { "ReturnCode", "MutationTime", "Table", "TablesToDelete", "ColumnName", "FamilyName", "FunctionName", "Function", "SharedTablesToDelete", "Schema", "AutoPartitionNum", "ViewIndexId", });
+              new java.lang.String[] { "ReturnCode", "MutationTime", "Table", "TablesToDelete", "ColumnName", "FamilyName", "FunctionName", "Function", "SharedTablesToDelete", "Schema", "AutoPartitionNum", "ViewIndexId", "ViewIndexType", });
           internal_static_GetTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(2);
           internal_static_GetTableRequest_fieldAccessorTable = new


[28/50] [abbrv] phoenix git commit: PHOENIX-4835 LoggingPhoenixConnection should log metrics upon connection close

Posted by ja...@apache.org.
PHOENIX-4835 LoggingPhoenixConnection should log metrics upon connection close


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

Branch: refs/heads/omid2
Commit: eb79c5b11b95bcd261c150ecd0f709712be2217c
Parents: dbbb112
Author: Karan Mehta <ka...@gmail.com>
Authored: Thu Aug 16 15:08:12 2018 -0700
Committer: Karan Mehta <ka...@gmail.com>
Committed: Fri Aug 17 12:02:22 2018 -0700

----------------------------------------------------------------------
 .../monitoring/BasePhoenixMetricsIT.java        | 128 +++++++++++++
 .../monitoring/PhoenixLoggingMetricsIT.java     | 181 +++++++++++++++++++
 .../phoenix/monitoring/PhoenixMetricsIT.java    | 170 ++---------------
 .../phoenix/jdbc/LoggingPhoenixConnection.java  |  12 +-
 4 files changed, 332 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb79c5b1/phoenix-core/src/it/java/org/apache/phoenix/monitoring/BasePhoenixMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/BasePhoenixMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/BasePhoenixMetricsIT.java
new file mode 100644
index 0000000..5c016f6
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/BasePhoenixMetricsIT.java
@@ -0,0 +1,128 @@
+/*
+ * 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 com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.phoenix.monitoring.MetricType.SCAN_BYTES;
+import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTED_COUNTER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class BasePhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
+
+    static final int MAX_RETRIES = 5;
+
+    static final List<MetricType> mutationMetricsToSkip =
+    Lists.newArrayList(MetricType.MUTATION_COMMIT_TIME);
+    static final List<MetricType> readMetricsToSkip =
+    Lists.newArrayList(MetricType.TASK_QUEUE_WAIT_TIME,
+            MetricType.TASK_EXECUTION_TIME, MetricType.TASK_END_TO_END_TIME,
+            MetricType.COUNT_MILLS_BETWEEN_NEXTS);
+    static final String CUSTOM_URL_STRING = "SESSION";
+    static final AtomicInteger numConnections = new AtomicInteger(0);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        // Phoenix Global client metrics are enabled by default
+        // 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);
+
+    }
+
+    Connection insertRowsInTable(String tableName, long numRows) throws SQLException {
+        String dml = "UPSERT INTO " + tableName + " VALUES (?, ?)";
+        Connection conn = DriverManager.getConnection(getUrl());
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        for (int i = 1; i <= numRows; i++) {
+            stmt.setString(1, "key" + i);
+            stmt.setString(2, "value" + i);
+            stmt.executeUpdate();
+        }
+        conn.commit();
+        return conn;
+    }
+
+    void assertReadMetricsForMutatingSql(String tableName, long tableSaltBuckets,
+                                                 Map<String, Map<MetricType, Long>> readMetrics) {
+        assertTrue("No read metrics present when there should have been!", readMetrics.size() > 0);
+        int numTables = 0;
+        for (Map.Entry<String, Map<MetricType, Long>> entry : readMetrics.entrySet()) {
+            String t = entry.getKey();
+            assertEquals("Table name didn't match for read metrics", tableName, t);
+            numTables++;
+            Map<MetricType, Long> p = entry.getValue();
+            assertTrue("No read metrics present when there should have been", p.size() > 0);
+            for (Map.Entry<MetricType, Long> metric : p.entrySet()) {
+                MetricType metricType = metric.getKey();
+                long metricValue = metric.getValue();
+                if (metricType.equals(TASK_EXECUTED_COUNTER)) {
+                    assertEquals(tableSaltBuckets, metricValue);
+                } else if (metricType.equals(SCAN_BYTES)) {
+                    assertTrue("Scan bytes read should be greater than zero", metricValue > 0);
+                }
+            }
+        }
+        assertEquals("There should have been read metrics only for one table: " + tableName, 1, numTables);
+    }
+
+    void assertMutationMetrics(String tableName, int numRows, Map<String, Map<MetricType, Long>> mutationMetrics) {
+        assertTrue("No mutation metrics present when there should have been", mutationMetrics.size() > 0);
+        for (Map.Entry<String, Map<MetricType, Long>> entry : mutationMetrics.entrySet()) {
+            String t = entry.getKey();
+            assertEquals("Table name didn't match for mutation metrics", tableName, t);
+            Map<MetricType, Long> p = entry.getValue();
+            assertEquals("There should have been four metrics", 4, p.size());
+            for (Map.Entry<MetricType, Long> metric : p.entrySet()) {
+                MetricType metricType = metric.getKey();
+                long metricValue = metric.getValue();
+                if (metricType.equals(MetricType.MUTATION_BATCH_SIZE)) {
+                    assertEquals("Mutation batch sizes didn't match!", numRows, metricValue);
+                } else if (metricType.equals(MetricType.MUTATION_COMMIT_TIME)) {
+                    assertTrue("Mutation commit time should be greater than zero", metricValue > 0);
+                } else if (metricType.equals(MetricType.MUTATION_BYTES)) {
+                    assertTrue("Mutation bytes size should be greater than zero", metricValue > 0);
+                } else if (metricType.equals(MetricType.MUTATION_BATCH_FAILED_SIZE)) {
+                    assertEquals("Zero failed mutations expected", 0, metricValue);
+                }
+            }
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb79c5b1/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
new file mode 100644
index 0000000..02640e7
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixLoggingMetricsIT.java
@@ -0,0 +1,181 @@
+/*
+ * 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 com.google.common.collect.Maps;
+import org.apache.phoenix.jdbc.LoggingPhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixMetricsLog;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+
+public class PhoenixLoggingMetricsIT extends BasePhoenixMetricsIT {
+
+    private static final int NUM_ROWS = 10;
+
+    private final Map<MetricType, Long> overAllQueryMetricsMap = Maps.newHashMap();
+    private final Map<String, Map<MetricType, Long>> requestReadMetricsMap = Maps.newHashMap();
+    private final Map<String, Map<MetricType, Long>> mutationWriteMetricsMap = Maps.newHashMap();
+    private final Map<String, Map<MetricType, Long>> mutationReadMetricsMap = Maps.newHashMap();
+
+    private String tableName1;
+    private String tableName2;
+    private LoggingPhoenixConnection loggedConn;
+
+    @Before
+    public void beforeTest() throws Exception {
+        clearAllTestMetricMaps();
+        tableName1 = generateUniqueName();
+        String ddl = "CREATE TABLE " + tableName1 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
+        Connection setupConn = DriverManager.getConnection(getUrl());
+        setupConn.createStatement().execute(ddl);
+        setupConn.close();
+        insertRowsInTable(tableName1, NUM_ROWS);
+
+        tableName2 = generateUniqueName();
+        ddl = "CREATE TABLE " + tableName2 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
+        setupConn = DriverManager.getConnection(getUrl());
+        setupConn.createStatement().execute(ddl);
+        setupConn.close();
+
+        Connection testConn = DriverManager.getConnection(getUrl());
+        loggedConn = getLoggingPhoenixConnection(testConn);
+    }
+
+    @Test
+    public void testPhoenixMetricsLoggedOnCommit() throws Exception {
+        // run SELECT to verify read metrics are logged
+        String query = "SELECT * FROM " + tableName1;
+        Statement stmt = loggedConn.createStatement();
+        ResultSet rs = stmt.executeQuery(query);
+        while (rs.next()) {
+        }
+        rs.close();
+        assertTrue("Read metrics for not found for " + tableName1,
+                requestReadMetricsMap.get(tableName1).size() > 0);
+        assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
+
+        // run UPSERT SELECT to verify mutation metrics are logged
+        String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
+        loggedConn.createStatement().executeUpdate(upsertSelect);
+
+        // Assert that metrics are logged upon commit
+        loggedConn.commit();
+        assertTrue("Mutation write metrics for not found for " + tableName2,
+                mutationWriteMetricsMap.get(tableName2).size() > 0);
+        assertMutationMetrics(tableName2, NUM_ROWS, mutationWriteMetricsMap);
+        assertTrue("Mutation read metrics for not found for " + tableName1,
+                mutationReadMetricsMap.get(tableName1).size() > 0);
+        assertReadMetricsForMutatingSql(tableName1, 1, mutationReadMetricsMap);
+
+        clearAllTestMetricMaps();
+
+        // Assert that metrics logging happens only once
+        loggedConn.close();
+        assertTrue("Mutation write metrics are not logged again.",
+                mutationWriteMetricsMap.size() == 0);
+        assertTrue("Mutation read metrics are not logged again.",
+                mutationReadMetricsMap.size() == 0);
+
+        clearAllTestMetricMaps();
+
+        // Assert that metrics logging happens only once again
+        loggedConn.close();
+        assertTrue("Mutation write metrics are not logged again.",
+                mutationWriteMetricsMap.size() == 0);
+        assertTrue("Mutation read metrics are not logged again.",
+                mutationReadMetricsMap.size() == 0);
+    }
+
+    @Test
+    public void testPhoenixMetricsLoggedOnClose() throws Exception {
+        // run SELECT to verify read metrics are logged
+        String query = "SELECT * FROM " + tableName1;
+        Statement stmt = loggedConn.createStatement();
+        ResultSet rs = stmt.executeQuery(query);
+        while (rs.next()) {
+        }
+        rs.close();
+        assertTrue("Read metrics for not found for " + tableName1,
+                requestReadMetricsMap.get(tableName1).size() > 0);
+        assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
+
+        // run UPSERT SELECT to verify mutation metrics are logged
+        String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
+        loggedConn.createStatement().executeUpdate(upsertSelect);
+
+        // Autocommit is turned off by default
+        // Hence mutation metrics are not expected during connection close
+        loggedConn.close();
+        assertTrue("Mutation write metrics are not logged for " + tableName2,
+                mutationWriteMetricsMap.size() == 0);
+        assertTrue("Mutation read metrics for not found for " + tableName1,
+                mutationReadMetricsMap.get(tableName1).size() > 0);
+        assertReadMetricsForMutatingSql(tableName1, 1, mutationReadMetricsMap);
+
+        clearAllTestMetricMaps();
+
+        loggedConn.close();
+        assertTrue("Mutation write metrics are not logged again.",
+                mutationWriteMetricsMap.size() == 0);
+        assertTrue("Mutation read metrics are not logged again.",
+                mutationReadMetricsMap.size() == 0);
+    }
+
+    void clearAllTestMetricMaps() {
+        overAllQueryMetricsMap.clear();
+        requestReadMetricsMap.clear();
+        mutationWriteMetricsMap.clear();
+        mutationReadMetricsMap.clear();
+    }
+
+    LoggingPhoenixConnection getLoggingPhoenixConnection(Connection conn) {
+        return new LoggingPhoenixConnection(conn, new PhoenixMetricsLog() {
+            @Override
+            public void logOverAllReadRequestMetrics(
+                    Map<MetricType, Long> overAllQueryMetrics) {
+                overAllQueryMetricsMap.putAll(overAllQueryMetrics);
+            }
+
+            @Override
+            public void logRequestReadMetrics(
+                    Map<String, Map<MetricType, Long>> requestReadMetrics) {
+                requestReadMetricsMap.putAll(requestReadMetrics);
+            }
+
+            @Override
+            public void logWriteMetricsfoForMutations(
+                    Map<String, Map<MetricType, Long>> mutationWriteMetrics) {
+                mutationWriteMetricsMap.putAll(mutationWriteMetrics);
+            }
+
+            @Override
+            public void logReadMetricInfoForMutationsSinceLastReset(
+                    Map<String, Map<MetricType, Long>> mutationReadMetrics) {
+                mutationReadMetricsMap.putAll(mutationReadMetrics);
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb79c5b1/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
index 4c5c592..0882cec 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
@@ -34,7 +34,6 @@ import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_SPOOL_FIL
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_TASK_END_TO_END_TIME;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_TASK_EXECUTION_TIME;
 import static org.apache.phoenix.monitoring.MetricType.MEMORY_CHUNK_BYTES;
-import static org.apache.phoenix.monitoring.MetricType.SCAN_BYTES;
 import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTED_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTION_TIME;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
@@ -59,51 +58,34 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.phoenix.compile.StatementContext;
-import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.jdbc.LoggingPhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDriver;
-import org.apache.phoenix.jdbc.PhoenixMetricsLog;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.log.LogLevel;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
-
-    private static final List<MetricType> mutationMetricsToSkip =
-            Lists.newArrayList(MetricType.MUTATION_COMMIT_TIME);
-    private static final List<MetricType> readMetricsToSkip =
-            Lists.newArrayList(MetricType.TASK_QUEUE_WAIT_TIME,
-                MetricType.TASK_EXECUTION_TIME, MetricType.TASK_END_TO_END_TIME,
-                MetricType.COUNT_MILLS_BETWEEN_NEXTS);
-    private static final String CUSTOM_URL_STRING = "SESSION";
-    private static final AtomicInteger numConnections = new AtomicInteger(0);
-
-    @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);
-    }
+/**
+ * Tests that
+ * 1. Phoenix Global metrics are exposed via
+ *   a. PhoenixRuntime b. Hadoop-Metrics2 defined sinks
+ * 2. Phoenix Request level metrics are exposed via
+ *   a. PhoenixRuntime
+ */
+public class PhoenixMetricsIT extends BasePhoenixMetricsIT {
+
+    private static final Log LOG = LogFactory.getLog(PhoenixMetricsIT.class);
 
     @Test
     public void testResetGlobalPhoenixMetrics() {
@@ -740,19 +722,6 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
                 expectedTableNames.size() == 0);
     }
 
-    private Connection insertRowsInTable(String tableName, long numRows) throws SQLException {
-        String dml = "UPSERT INTO " + tableName + " VALUES (?, ?)";
-        Connection conn = DriverManager.getConnection(getUrl());
-        PreparedStatement stmt = conn.prepareStatement(dml);
-        for (int i = 1; i <= numRows; i++) {
-            stmt.setString(1, "key" + i);
-            stmt.setString(2, "value" + i);
-            stmt.executeUpdate();
-        }
-        conn.commit();
-        return conn;
-    }
-
     // number of records read should be number of bytes at the end
     public static final long SCAN_BYTES_DELTA = 1;
 
@@ -800,52 +769,6 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
         }
     }
 
-    private void assertReadMetricsForMutatingSql(String tableName, long tableSaltBuckets,
-            Map<String, Map<MetricType, Long>> readMetrics) {
-        assertTrue("No read metrics present when there should have been!", readMetrics.size() > 0);
-        int numTables = 0;
-        for (Entry<String, Map<MetricType, Long>> entry : readMetrics.entrySet()) {
-            String t = entry.getKey();
-            assertEquals("Table name didn't match for read metrics", tableName, t);
-            numTables++;
-            Map<MetricType, Long> p = entry.getValue();
-            assertTrue("No read metrics present when there should have been", p.size() > 0);
-            for (Entry<MetricType, Long> metric : p.entrySet()) {
-            	MetricType metricType = metric.getKey();
-                long metricValue = metric.getValue();
-                if (metricType.equals(TASK_EXECUTED_COUNTER)) {
-                    assertEquals(tableSaltBuckets, metricValue);
-                } else if (metricType.equals(SCAN_BYTES)) {
-                    assertTrue("Scan bytes read should be greater than zero", metricValue > 0);
-                }
-            }
-        }
-        assertEquals("There should have been read metrics only for one table: " + tableName, 1, numTables);
-    }
-
-    private void assertMutationMetrics(String tableName, int numRows, Map<String, Map<MetricType, Long>> mutationMetrics) {
-        assertTrue("No mutation metrics present when there should have been", mutationMetrics.size() > 0);
-        for (Entry<String, Map<MetricType, Long>> entry : mutationMetrics.entrySet()) {
-            String t = entry.getKey();
-            assertEquals("Table name didn't match for mutation metrics", tableName, t);
-            Map<MetricType, Long> p = entry.getValue();
-            assertEquals("There should have been four metrics", 4, p.size());
-            for (Entry<MetricType, Long> metric : p.entrySet()) {
-            	MetricType metricType = metric.getKey();
-                long metricValue = metric.getValue();
-                if (metricType.equals(MetricType.MUTATION_BATCH_SIZE)) {
-                    assertEquals("Mutation batch sizes didn't match!", numRows, metricValue);
-                } else if (metricType.equals(MetricType.MUTATION_COMMIT_TIME)) {
-                    assertTrue("Mutation commit time should be greater than zero", metricValue > 0);
-                } else if (metricType.equals(MetricType.MUTATION_BYTES)) {
-                    assertTrue("Mutation bytes size should be greater than zero", metricValue > 0);
-                } else if (metricType.equals(MetricType.MUTATION_BATCH_FAILED_SIZE)) {
-                    assertEquals("Zero failed mutations expected", 0, metricValue);
-                }
-            }
-        }
-    }
-
     @Test
     public void testGetConnectionsForSameUrlConcurrently()  throws Exception {
         // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
@@ -1020,74 +943,5 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
             return c;
         }
     }
-    
-    @Test
-    public void testPhoenixMetricsLogged() throws Exception {
-        final Map<MetricType, Long> overAllQueryMetricsMap = Maps.newHashMap();
-        final Map<String, Map<MetricType, Long>> requestReadMetricsMap = Maps.newHashMap();
-        final Map<String, Map<MetricType, Long>> mutationWriteMetricsMap = Maps.newHashMap();
-        final Map<String, Map<MetricType, Long>> mutationReadMetricsMap = Maps.newHashMap();
-
-        String tableName1 = generateUniqueName();
-        String ddl = "CREATE TABLE " + tableName1 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
-        Connection ddlConn = DriverManager.getConnection(getUrl());
-        ddlConn.createStatement().execute(ddl);
-        ddlConn.close();
-        insertRowsInTable(tableName1, 10);
-
-        String tableName2 = generateUniqueName();
-        ddl = "CREATE TABLE " + tableName2 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
-        ddlConn = DriverManager.getConnection(getUrl());
-        ddlConn.createStatement().execute(ddl);
-        ddlConn.close();
 
-        Connection conn = DriverManager.getConnection(getUrl());
-        LoggingPhoenixConnection protectedConn =
-                new LoggingPhoenixConnection(conn, new PhoenixMetricsLog() {
-                    @Override
-                    public void logOverAllReadRequestMetrics(
-                            Map<MetricType, Long> overAllQueryMetrics) {
-                        overAllQueryMetricsMap.putAll(overAllQueryMetrics);
-                    }
-
-                    @Override
-                    public void logRequestReadMetrics(
-                            Map<String, Map<MetricType, Long>> requestReadMetrics) {
-                        requestReadMetricsMap.putAll(requestReadMetrics);
-                    }
-
-                    @Override
-                    public void logWriteMetricsfoForMutations(
-                            Map<String, Map<MetricType, Long>> mutationWriteMetrics) {
-                        mutationWriteMetricsMap.putAll(mutationWriteMetrics);
-                    }
-
-                    @Override
-                    public void logReadMetricInfoForMutationsSinceLastReset(
-                            Map<String, Map<MetricType, Long>> mutationReadMetrics) {
-                        mutationReadMetricsMap.putAll(mutationReadMetrics);
-                    }
-                });
-        
-        // run SELECT to verify read metrics are logged
-        String query = "SELECT * FROM " + tableName1;
-        Statement stmt = protectedConn.createStatement();
-        ResultSet rs = stmt.executeQuery(query);
-        while (rs.next()) {
-        }
-        rs.close();
-        assertTrue("Read metrics for not found for " + tableName1,
-            requestReadMetricsMap.get(tableName1).size() > 0);
-        assertTrue("Overall read metrics for not found ", overAllQueryMetricsMap.size() > 0);
-
-        // run UPSERT SELECT to verify mutation metrics are logged
-        String upsertSelect = "UPSERT INTO " + tableName2 + " SELECT * FROM " + tableName1;
-        protectedConn.createStatement().executeUpdate(upsertSelect);
-        protectedConn.commit();
-        assertTrue("Mutation write metrics for not found for " + tableName2,
-            mutationWriteMetricsMap.get(tableName2).size() > 0);
-        assertTrue("Mutation read metrics for not found for " + tableName1,
-            mutationReadMetricsMap.get(tableName1).size() > 0);
-        protectedConn.close();
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb79c5b1/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
index d98da83..9a2e00f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/LoggingPhoenixConnection.java
@@ -98,7 +98,7 @@ public class LoggingPhoenixConnection extends DelegateConnection {
         return new LoggingPhoenixPreparedStatement(super.prepareStatement(sql, columnNames),
                 phoenixMetricsLog);
     }
-    
+
     @Override
     public void commit() throws SQLException {
         super.commit();
@@ -107,4 +107,14 @@ public class LoggingPhoenixConnection extends DelegateConnection {
         PhoenixRuntime.resetMetrics(conn);
     }
 
+    @Override
+    public void close() throws SQLException {
+        try {
+            phoenixMetricsLog.logWriteMetricsfoForMutations(PhoenixRuntime.getWriteMetricInfoForMutationsSinceLastReset(conn));
+            phoenixMetricsLog.logReadMetricInfoForMutationsSinceLastReset(PhoenixRuntime.getReadMetricInfoForMutationsSinceLastReset(conn));
+            PhoenixRuntime.resetMetrics(conn);
+        } finally {
+            super.close();
+        }
+    }
 }


[48/50] [abbrv] phoenix git commit: PHOENIX-4907 - IndexScrutinyTool should use empty catalog instead of null

Posted by ja...@apache.org.
PHOENIX-4907 - IndexScrutinyTool should use empty catalog instead of null


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

Branch: refs/heads/omid2
Commit: 7fa11ff62412c57bde2ddc8037dddd248d73341e
Parents: d8a78e1
Author: Geoffrey <gj...@apache.org>
Authored: Mon Sep 17 16:09:44 2018 -0700
Committer: Geoffrey <gj...@apache.org>
Committed: Tue Sep 18 13:01:30 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7fa11ff6/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
index f3ff39e..d9a14bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyTool.java
@@ -499,7 +499,7 @@ public class IndexScrutinyTool extends Configured implements Tool {
 
         ResultSet rs = null;
         try {
-            rs = dbMetaData.getIndexInfo(null, schemaName, tableName, false, false);
+            rs = dbMetaData.getIndexInfo("", schemaName, tableName, false, false);
             while (rs.next()) {
                 final String indexName = rs.getString(6);
                 if (indexTable.equalsIgnoreCase(indexName)) {


[45/50] [abbrv] phoenix git commit: PHOENIX-4884 Update INSTR to handle literals and non-literals in either function argument

Posted by ja...@apache.org.
PHOENIX-4884 Update INSTR to handle literals and non-literals in either function argument

INSTR previously only handled arguments of the form non-literal and literal, but the documentation
doesn't clearly state this. We can support all variants though.


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

Branch: refs/heads/omid2
Commit: 3b03e1b0bddaf3b52637187ff556fbb39b1dddef
Parents: 012bb31
Author: Josh Elser <el...@apache.org>
Authored: Fri Aug 31 10:59:47 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Sep 10 22:14:05 2018 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/InstrFunctionIT.java | 35 +++++++++
 .../expression/function/InstrFunction.java      | 78 +++++++++++++-------
 .../expression/function/InstrFunctionTest.java  | 44 +++++++----
 3 files changed, 114 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3b03e1b0/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
index 270b1ec..bc86980 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
@@ -131,4 +131,39 @@ public class InstrFunctionIT extends ParallelStatsDisabledIT {
         testInstrFilter(conn, queryToExecute,"abcdefghijkl");
     }
 
+    @Test
+    public void testNonLiteralExpression() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateUniqueName();
+        initTable(conn, tableName, "ASC", "asdf", "sdf");
+        // Should be able to use INSTR with a non-literal expression as the 2nd argument
+        String query = "SELECT INSTR(name, substr) FROM " + tableName;
+        testInstr(conn, query, 2);
+    }
+
+    @Test
+    public void testNonLiteralSourceExpression() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateUniqueName();
+        initTable(conn, tableName, "ASC", "asdf", "sdf");
+        // Using the function inside the SELECT will test client-side.
+        String query = "SELECT INSTR('asdf', 'sdf') FROM " + tableName;
+        testInstr(conn, query, 2);
+        query = "SELECT INSTR('asdf', substr) FROM " + tableName;
+        testInstr(conn, query, 2);
+        query = "SELECT INSTR('qwerty', 'sdf') FROM " + tableName;
+        testInstr(conn, query, 0);
+        query = "SELECT INSTR('qwerty', substr) FROM " + tableName;
+        testInstr(conn, query, 0);
+        // Test the built-in function in a where clause to make sure
+        // it works server-side (and not just client-side).
+        query = "SELECT name FROM " + tableName + " WHERE INSTR(name, substr) = 2";
+        testInstrFilter(conn, query, "asdf");
+        query = "SELECT name FROM " + tableName + " WHERE INSTR(name, 'sdf') = 2";
+        testInstrFilter(conn, query, "asdf");
+        query = "SELECT name FROM " + tableName + " WHERE INSTR('asdf', substr) = 2";
+        testInstrFilter(conn, query, "asdf");
+        query = "SELECT name FROM " + tableName + " WHERE INSTR('asdf', 'sdf') = 2";
+        testInstrFilter(conn, query, "asdf");
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3b03e1b0/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InstrFunction.java
index 7a002f8..e6b4c16 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InstrFunction.java
@@ -30,7 +30,6 @@ 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.PVarchar;
-import org.apache.phoenix.util.ByteUtil;
 
 @BuiltInFunction(name=InstrFunction.NAME, args={
         @Argument(allowedTypes={ PVarchar.class }),
@@ -38,8 +37,9 @@ import org.apache.phoenix.util.ByteUtil;
 public class InstrFunction extends ScalarFunction{
     
     public static final String NAME = "INSTR";
-    
-    private String strToSearch = null;
+
+    private String literalSourceStr = null;
+    private String literalSearchStr = null;
     
     public InstrFunction() { }
     
@@ -49,40 +49,62 @@ public class InstrFunction extends ScalarFunction{
     }
     
     private void init() {
-        Expression strToSearchExpression = getChildren().get(1);
-        if (strToSearchExpression instanceof LiteralExpression) {
-            Object strToSearchValue = ((LiteralExpression) strToSearchExpression).getValue();
-            if (strToSearchValue != null) {
-                this.strToSearch = strToSearchValue.toString();
-            }
+        literalSourceStr = maybeExtractLiteralString(getChildren().get(0));
+        literalSearchStr = maybeExtractLiteralString(getChildren().get(1));
+    }
+
+    /**
+     * Extracts the string-representation of {@code expr} only if {@code expr} is a
+     * non-null {@link LiteralExpression}.
+     *
+     * @param expr An Expression.
+     * @return The string value for the expression or null
+     */
+    private String maybeExtractLiteralString(Expression expr) {
+        if (expr instanceof LiteralExpression) {
+            // Whether the value is null or non-null, we can give it back right away
+            return (String) ((LiteralExpression) expr).getValue();
         }
+        return null;
     }
         
     
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
-        Expression child = getChildren().get(0);
-        
-        if (!child.evaluate(tuple, ptr)) {
-            return false;
-        }
-        
-        if (ptr.getLength() == 0) {
-            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-            return true;
+        String sourceStr = literalSourceStr;
+        if (sourceStr == null) {
+            Expression child = getChildren().get(0);
+
+            if (!child.evaluate(tuple, ptr)) {
+                return false;
+            }
+
+            // We need something non-empty to search against
+            if (ptr.getLength() == 0) {
+              return true;
+            }
+
+            sourceStr = (String) PVarchar.INSTANCE.toObject(ptr, child.getSortOrder());
         }
-        
-        int position;
-        //Logic for Empty string search
-        if (strToSearch == null){
-            position = 0;
-            ptr.set(PInteger.INSTANCE.toBytes(position));
-            return true;
+
+        String searchStr = literalSearchStr;
+        // A literal was not provided, try to evaluate the expression to a literal
+        if (searchStr == null){
+            Expression child = getChildren().get(1);
+
+            if (!child.evaluate(tuple, ptr)) {
+              return false;
+            }
+
+            // A null (or zero-length) search string
+            if (ptr.getLength() == 0) {
+              return true;
+            }
+            
+            searchStr = (String) PVarchar.INSTANCE.toObject(ptr, child.getSortOrder());
         }
-        
-        String sourceStr = (String) PVarchar.INSTANCE.toObject(ptr, getChildren().get(0).getSortOrder());
 
-        position = sourceStr.indexOf(strToSearch) + 1;
+        int position = sourceStr.indexOf(searchStr) + 1;
         ptr.set(PInteger.INSTANCE.toBytes(position));
         return true;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3b03e1b0/phoenix-core/src/test/java/org/apache/phoenix/expression/function/InstrFunctionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/InstrFunctionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/InstrFunctionTest.java
index 359d772..6fd16ec 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/InstrFunctionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/InstrFunctionTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.expression.function;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.SQLException;
@@ -32,18 +34,27 @@ import org.apache.phoenix.schema.types.PVarchar;
 import org.junit.Test;
 
 public class InstrFunctionTest {
+
+    private static Object evaluateExpression(String value, PDataType<?> dataType, String strToSearch, SortOrder order) throws SQLException {
+      Expression inputArg = LiteralExpression.newConstant(value,dataType,order);
+
+      Expression strToSearchExp = LiteralExpression.newConstant(strToSearch,dataType);
+      List<Expression> expressions = Arrays.<Expression>asList(inputArg,strToSearchExp);
+      Expression instrFunction = new InstrFunction(expressions);
+      ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+      instrFunction.evaluate(null,ptr);
+      return instrFunction.getDataType().toObject(ptr);
+    }
     
-    public static void inputExpression(String value, PDataType dataType, String strToSearch,Integer expected, SortOrder order) throws SQLException{
-        Expression inputArg = LiteralExpression.newConstant(value,dataType,order);
-        
-        Expression strToSearchExp = LiteralExpression.newConstant(strToSearch,dataType);
-        List<Expression> expressions = Arrays.<Expression>asList(inputArg,strToSearchExp);
-        Expression instrFunction = new InstrFunction(expressions);
-        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-        instrFunction.evaluate(null,ptr);
-        Integer result = (Integer) instrFunction.getDataType().toObject(ptr);
-        assertTrue(result.compareTo(expected) == 0);
-        
+    public static void inputExpression(String value, PDataType<?> dataType, String strToSearch,Integer expected, SortOrder order) throws SQLException {
+        Object obj = evaluateExpression(value, dataType, strToSearch, order);
+        assertNotNull("Result was unexpectedly null", obj);
+        assertTrue(((Integer) obj).compareTo(expected) == 0);
+    }
+
+    public static void inputNullExpression(String value, PDataType<?> dataType, String strToSearch, SortOrder order) throws SQLException {
+        Object obj = evaluateExpression(value, dataType, strToSearch, order);
+        assertNull("Result was unexpectedly non-null", obj);
     }
     
     
@@ -72,10 +83,13 @@ public class InstrFunctionTest {
         inputExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, " ", 6, SortOrder.ASC);
         
         inputExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, " ", 6, SortOrder.DESC);
-        
-        inputExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, "", 0, SortOrder.ASC);
-        
-        inputExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, "", 0, SortOrder.DESC);
+
+        // Phoenix can't represent empty strings, so an empty or null search string should return null
+        // See PHOENIX-4884 for more chatter.
+        inputNullExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, "", SortOrder.ASC);
+        inputNullExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, "", SortOrder.DESC);
+        inputNullExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, null, SortOrder.ASC);
+        inputNullExpression("ABCDE FGHiJKL",PVarchar.INSTANCE, null, SortOrder.DESC);
         
         inputExpression("ABCDEABC",PVarchar.INSTANCE, "ABC", 1, SortOrder.ASC);
         


[27/50] [abbrv] phoenix git commit: PHOENIX-4750 Resolve server customizers and provide them to Avatica

Posted by ja...@apache.org.
PHOENIX-4750 Resolve server customizers and provide them to Avatica

Resolve server customizers on the PQS classpath and provide them to the
HttpServer builder.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: dbbb1125d1a219b3dec9250f80734410b7a1a8c8
Parents: fe4c053
Author: Alex Araujo <al...@gmail.com>
Authored: Mon Jun 4 16:32:10 2018 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Thu Aug 16 10:41:11 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java     |   1 +
 .../phoenix/end2end/QueryServerTestUtil.java    | 187 +++++++++++++++++++
 .../phoenix/end2end/ServerCustomizersIT.java    | 147 +++++++++++++++
 .../phoenix/queryserver/server/QueryServer.java |  26 ++-
 .../server/ServerCustomizersFactory.java        |  49 +++++
 .../server/ServerCustomizersTest.java           |  87 +++++++++
 7 files changed, 496 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/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 2bb9350..d290174 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
@@ -251,6 +251,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_SPNEGO_AUTH_DISABLED_ATTRIB = "phoenix.queryserver.spnego.auth.disabled";
     public static final String QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR_ATTRIB = "phoenix.queryserver.withRemoteUserExtractor";
     public static final String QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "phoenix.queryserver.remoteUserExtractor.param";
+    public static final String QUERY_SERVER_CUSTOMIZERS_ENABLED = "phoenix.queryserver.customizers.enabled";
     public static final String QUERY_SERVER_DISABLE_KERBEROS_LOGIN = "phoenix.queryserver.disable.kerberos.login";
 
     // metadata configs

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/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 4be8f81..c68e793 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
@@ -301,6 +301,7 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR = false;
     public static final String DEFAULT_QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "doAs";
     public static final boolean DEFAULT_QUERY_SERVER_DISABLE_KERBEROS_LOGIN = false;
+    public static final boolean DEFAULT_QUERY_SERVER_CUSTOMIZERS_ENABLED = false;
 
     public static final boolean DEFAULT_RENEW_LEASE_ENABLED = true;
     public static final int DEFAULT_RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java
new file mode 100644
index 0000000..01f73ae
--- /dev/null
+++ b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java
@@ -0,0 +1,187 @@
+/*
+ * 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 java.io.File;
+import java.security.PrivilegedAction;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.LocalHBaseCluster;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.phoenix.query.ConfigurationFactory;
+import org.apache.phoenix.queryserver.client.ThinClientUtil;
+import org.apache.phoenix.queryserver.server.QueryServer;
+import org.apache.phoenix.util.InstanceResolver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
+
+public class QueryServerTestUtil {
+    private static final Logger LOG = LoggerFactory.getLogger(QueryServerTestUtil.class);
+
+    private final Configuration conf;
+    private final HBaseTestingUtility util;
+    private LocalHBaseCluster hbase;
+
+    private final QueryServer pqs;
+    private int port;
+    private String url;
+
+    private String principal;
+    private File keytab;
+
+    private ExecutorService executor;
+
+    public QueryServerTestUtil(Configuration conf) {
+        this.conf = Objects.requireNonNull(conf);
+        this.util = new HBaseTestingUtility(conf);
+        this.pqs = new QueryServer(new String[0], conf);
+    }
+
+    public QueryServerTestUtil(Configuration conf, String principal, File keytab) {
+        this.conf = Objects.requireNonNull(conf);
+        this.principal = principal;
+        this.keytab = keytab;
+        this.util = new HBaseTestingUtility(conf);
+        this.pqs = new QueryServer(new String[0], conf);
+    }
+
+    public void startLocalHBaseCluster(Class testClass) throws Exception {
+        startLocalHBaseCluster(testClass.getCanonicalName());
+    }
+
+    public void startLocalHBaseCluster(String uniqueName) throws Exception {
+        LOG.debug("Starting local HBase cluster for '{}'", uniqueName);
+        // Start ZK
+        util.startMiniZKCluster();
+        // Start HDFS
+        util.startMiniDFSCluster(1);
+        // Start HBase
+        Path rootdir = util.getDataTestDirOnTestFS(uniqueName);
+        FSUtils.setRootDir(conf, rootdir);
+        hbase = new LocalHBaseCluster(conf, 1);
+        hbase.startup();
+    }
+
+    public void stopLocalHBaseCluster() throws Exception {
+        LOG.debug("Stopping local HBase cluster");
+        if (hbase != null) {
+            hbase.shutdown();
+            hbase.join();
+        }
+        if (util != null) {
+            util.shutdownMiniDFSCluster();
+            util.shutdownMiniZKCluster();
+        }
+    }
+
+    public void startQueryServer() throws Exception {
+        setupQueryServerConfiguration(conf);
+        executor = Executors.newSingleThreadExecutor();
+        if (!Strings.isNullOrEmpty(principal) && null != keytab) {
+            // Get the PQS ident for PQS to use
+            final UserGroupInformation ugi = UserGroupInformation
+                    .loginUserFromKeytabAndReturnUGI(principal, keytab.getAbsolutePath());
+            // Launch PQS, doing in the Kerberos login instead of letting PQS do it itself (which would
+            // break the HBase/HDFS logins also running in the same test case).
+            executor.submit(new Runnable() {
+                @Override
+                public void run() {
+                    ugi.doAs(new PrivilegedAction<Void>() {
+                        @Override
+                        public Void run() {
+                            pqs.run();
+                            return null;
+                        }
+                    });
+                }
+            });
+        } else {
+            // Launch PQS without a login
+            executor.submit(new Runnable() {
+                @Override
+                public void run() {
+                    pqs.run();
+                }
+            });
+        }
+        pqs.awaitRunning();
+        port = pqs.getPort();
+        url = ThinClientUtil.getConnectionUrl("localhost", port);
+    }
+
+    public void stopQueryServer() throws Exception {
+        if (pqs != null) {
+            pqs.stop();
+        }
+        if (executor != null) {
+            executor.shutdown();
+            if (!executor.awaitTermination(5, TimeUnit.SECONDS)) {
+                LOG.info("PQS didn't exit in 5 seconds, proceeding anyways.");
+            }
+        }
+    }
+
+    public static void setupQueryServerConfiguration(final Configuration conf) {
+        // Make sure the ConnectionInfo doesn't try to pull a default Configuration
+        InstanceResolver.getSingleton(ConfigurationFactory.class, new ConfigurationFactory() {
+            @Override
+            public Configuration getConfiguration() {
+                return conf;
+            }
+            @Override
+            public Configuration getConfiguration(Configuration confToClone) {
+                Configuration copy = new Configuration(conf);
+                copy.addResource(confToClone);
+                return copy;
+            }
+        });
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    public String getUrl() {
+        return url;
+    }
+
+    /**
+     * Returns the query server URL with the specified URL params
+     * @param params URL params
+     * @return URL with params
+     */
+    public String getUrl(Map<String, String> params) {
+        if (params == null || params.size() == 0) {
+            return url;
+        }
+        StringBuilder urlParams = new StringBuilder();
+        for (Map.Entry<String, String> param : params.entrySet()) {
+            urlParams.append(";").append(param.getKey()).append("=").append(param.getValue());
+        }
+        return url + urlParams;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
new file mode 100644
index 0000000..d990adb
--- /dev/null
+++ b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
@@ -0,0 +1,147 @@
+/*
+ * 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 java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.avatica.server.ServerCustomizer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.queryserver.server.ServerCustomizersFactory;
+import org.apache.phoenix.util.InstanceResolver;
+import org.eclipse.jetty.security.ConstraintMapping;
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.security.HashLoginService;
+import org.eclipse.jetty.security.authentication.BasicAuthenticator;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.util.security.Constraint;
+import org.eclipse.jetty.util.security.Credential;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ServerCustomizersIT extends BaseHBaseManagedTimeIT {
+    private static final Logger LOG = LoggerFactory.getLogger(ServerCustomizersIT.class);
+    private static final String USER_AUTHORIZED = "user3";
+    private static final String USER_NOT_AUTHORIZED = "user1";
+    private static final String USER_PW = "s3cr3t";
+
+    private static QueryServerTestUtil PQS_UTIL;
+
+    @Rule
+    public ExpectedException expected = ExpectedException.none();
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        Configuration conf = getTestClusterConfig();
+        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        PQS_UTIL = new QueryServerTestUtil(conf);
+        PQS_UTIL.startLocalHBaseCluster(ServerCustomizersIT.class);
+        // Register a test jetty server customizer
+        InstanceResolver.clearSingletons();
+        InstanceResolver.getSingleton(ServerCustomizersFactory.class, new ServerCustomizersFactory() {
+            @Override
+            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+                return Collections.<ServerCustomizer<Server>>singletonList(new TestServerCustomizer());
+            }
+        });
+        PQS_UTIL.startQueryServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception {
+        // Remove custom singletons for future tests
+        InstanceResolver.clearSingletons();
+        if (PQS_UTIL != null) {
+            PQS_UTIL.stopQueryServer();
+            PQS_UTIL.stopLocalHBaseCluster();
+        }
+    }
+
+    @Test
+    public void testUserAuthorized() throws Exception {
+        try (Connection conn = DriverManager.getConnection(PQS_UTIL.getUrl(
+                getBasicAuthParams(USER_AUTHORIZED)));
+                Statement stmt = conn.createStatement()) {
+            Assert.assertFalse("user3 should have access", stmt.execute(
+                "create table "+ServerCustomizersIT.class.getSimpleName()+" (pk integer not null primary key)"));
+        }
+    }
+
+    @Test
+    public void testUserNotAuthorized() throws Exception {
+        expected.expect(RuntimeException.class);
+        expected.expectMessage("HTTP/401");
+        try (Connection conn = DriverManager.getConnection(PQS_UTIL.getUrl(
+                getBasicAuthParams(USER_NOT_AUTHORIZED)));
+                Statement stmt = conn.createStatement()) {
+            Assert.assertFalse(stmt.execute(
+                    "select access from database"));
+        }
+    }
+
+    private Map<String, String> getBasicAuthParams(String user) {
+        Map<String, String> params = new HashMap<>();
+        params.put("authentication", "BASIC");
+        params.put("avatica_user", user);
+        params.put("avatica_password", USER_PW);
+        return params;
+    }
+
+    /**
+     * Contrived customizer that enables BASIC auth for a single user
+     */
+    public static class TestServerCustomizer implements ServerCustomizer<Server> {
+        @Override
+        public void customize(Server server) {
+            LOG.debug("Customizing server to allow requests for {}", USER_AUTHORIZED);
+            HashLoginService login = new HashLoginService();
+            login.putUser(USER_AUTHORIZED, Credential.getCredential(USER_PW), new String[] {"users"});
+            login.setName("users");
+
+            Constraint constraint = new Constraint();
+            constraint.setName(Constraint.__BASIC_AUTH);
+            constraint.setRoles(new String[]{"users"});
+            constraint.setAuthenticate(true);
+
+            ConstraintMapping cm = new ConstraintMapping();
+            cm.setConstraint(constraint);
+            cm.setPathSpec("/*");
+
+            ConstraintSecurityHandler security = new ConstraintSecurityHandler();
+            security.setAuthenticator(new BasicAuthenticator());
+            security.setRealmName("users");
+            security.addConstraintMapping(cm);
+            security.setLoginService(login);
+
+            // chain the PQS handler to security
+            security.setHandler(server.getHandlers()[0]);
+            server.setHandler(security);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/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 e3f0f52..47466c8 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
@@ -33,6 +33,7 @@ import org.apache.calcite.avatica.server.RemoteUserExtractor;
 import org.apache.calcite.avatica.server.RemoteUserExtractionException;
 import org.apache.calcite.avatica.server.HttpRequestRemoteUserExtractor;
 import org.apache.calcite.avatica.server.HttpQueryStringParameterRemoteUserExtractor;
+import org.apache.calcite.avatica.server.ServerCustomizer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -52,6 +53,7 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.loadbalancer.service.LoadBalanceZookeeperConf;
 import org.apache.phoenix.queryserver.register.Registry;
 import org.apache.phoenix.util.InstanceResolver;
+import org.eclipse.jetty.server.Server;
 
 import java.io.File;
 import java.io.IOException;
@@ -61,6 +63,7 @@ import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
@@ -228,14 +231,15 @@ public final class QueryServer extends Configured implements Tool, Runnable {
       Service service = new LocalService(meta);
 
       // Start building the Avatica HttpServer
-      final HttpServer.Builder builder = new HttpServer.Builder().withPort(port)
-          .withHandler(service, getSerialization(getConf()));
+      final HttpServer.Builder<Server> builder = HttpServer.Builder.<Server> newBuilder()
+          .withPort(port).withHandler(service, getSerialization(getConf()));
 
       // Enable client auth when using Kerberos auth for HBase
       if (isKerberos) {
         configureClientAuthentication(builder, disableSpnego);
       }
       setRemoteUserExtractorIfNecessary(builder, getConf());
+      enableServerCustomizersIfNecessary(builder, getConf());
 
       // Build and start the HttpServer
       server = builder.build();
@@ -405,12 +409,30 @@ public final class QueryServer extends Configured implements Tool, Runnable {
     new RemoteUserExtractorFactory.RemoteUserExtractorFactoryImpl();
 
   @VisibleForTesting
+  public void enableServerCustomizersIfNecessary(HttpServer.Builder<Server> builder, Configuration conf) {
+    if (conf.getBoolean(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED,
+            QueryServicesOptions.DEFAULT_QUERY_SERVER_CUSTOMIZERS_ENABLED)) {
+      builder.withServerCustomizers(createServerCustomizers(conf), Server.class);
+    }
+  }
+
+  private static final ServerCustomizersFactory DEFAULT_SERVER_CUSTOMIZERS =
+    new ServerCustomizersFactory.ServerCustomizersFactoryImpl();
+
+  @VisibleForTesting
   RemoteUserExtractor createRemoteUserExtractor(Configuration conf) {
     RemoteUserExtractorFactory factory =
         InstanceResolver.getSingleton(RemoteUserExtractorFactory.class, DEFAULT_USER_EXTRACTOR);
     return factory.createRemoteUserExtractor(conf);
   }
 
+  @VisibleForTesting
+  List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+    ServerCustomizersFactory factory =
+      InstanceResolver.getSingleton(ServerCustomizersFactory.class, DEFAULT_SERVER_CUSTOMIZERS);
+    return factory.createServerCustomizers(conf);
+  }
+
   /**
    * Use the correctly way to extract end user.
    */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
new file mode 100644
index 0000000..462cd5d
--- /dev/null
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.queryserver.server;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.avatica.server.ServerCustomizer;
+import org.apache.hadoop.conf.Configuration;
+import org.eclipse.jetty.server.Server;
+
+/**
+ * Creates customizers for the underlying Avatica HTTP server.
+ * Allows for fine grained control of authentication, etc.
+ */
+public interface ServerCustomizersFactory {
+    /**
+     * Creates a list of customizers that will customize the server.
+     * @param conf Configuration to use
+     * @return List of server suctomizers
+     */
+    List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf);
+
+    /**
+     * Factory that creates an empty list of customizers.
+     */
+    class ServerCustomizersFactoryImpl implements ServerCustomizersFactory {
+        private static final List<ServerCustomizer<Server>> EMPTY_LIST = Collections.emptyList();
+        @Override
+        public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+            return EMPTY_LIST;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
new file mode 100644
index 0000000..45fec37
--- /dev/null
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.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.queryserver.server;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.ServerCustomizer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.InstanceResolver;
+import org.eclipse.jetty.server.Server;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class ServerCustomizersTest {
+    @Before @After
+    public void clearSingletons() {
+        // clean up singletons
+        InstanceResolver.clearSingletons();
+    }
+
+    @Test
+    public void testDefaultFactory() {
+        QueryServer queryServer = new QueryServer();
+        // the default factory creates an empty list of server customizers
+        List<ServerCustomizer<Server>> customizers =
+            queryServer.createServerCustomizers(new Configuration());
+        Assert.assertEquals(0, customizers.size());
+    }
+
+    @Test
+    public void testUseProvidedCustomizers() {
+        final List<ServerCustomizer<Server>> expected =
+            Collections.<ServerCustomizer<Server>> singletonList(new ServerCustomizer<Server>() {
+              @Override
+              public void customize(Server server) {
+                // no-op customizer
+              }
+        });
+        // Register the server customizer list
+        InstanceResolver.getSingleton(ServerCustomizersFactory.class, new ServerCustomizersFactory() {
+            @Override
+            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+                return expected;
+            }
+        });
+        Configuration conf = new Configuration(false);
+        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        QueryServer queryServer = new QueryServer();
+        List<ServerCustomizer<Server>> actual = queryServer.createServerCustomizers(conf);
+        Assert.assertEquals("Customizers are different", expected, actual);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testEnableCustomizers() {
+        HttpServer.Builder builder = mock(HttpServer.Builder.class);
+        Configuration conf = new Configuration(false);
+        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        QueryServer queryServer = new QueryServer();
+        queryServer.enableServerCustomizersIfNecessary(builder, conf);
+        verify(builder).withServerCustomizers(anyList(), any(Class.class));
+    }
+}
\ No newline at end of file


[11/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

Posted by ja...@apache.org.
PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)


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

Branch: refs/heads/omid2
Commit: 93fdd5bad22cde313c9f34fe7448dca44377a27c
Parents: 4cab4c2
Author: Thomas D'Silva <td...@apache.org>
Authored: Sat Jul 14 11:34:47 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Jul 18 21:46:59 2018 -0700

----------------------------------------------------------------------
 .../StatisticsCollectionRunTrackerIT.java       |    2 +-
 .../AlterMultiTenantTableWithViewsIT.java       |  284 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |   45 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  545 ++--
 .../phoenix/end2end/AppendOnlySchemaIT.java     |    4 +-
 .../end2end/BaseTenantSpecificViewIndexIT.java  |   38 +-
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |   69 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   38 +-
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   37 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |    9 +-
 .../apache/phoenix/end2end/SaltedViewIT.java    |   45 -
 .../phoenix/end2end/SplitSystemCatalogIT.java   |   80 +
 .../end2end/SplitSystemCatalogTests.java        |   11 +
 .../StatsEnabledSplitSystemCatalogIT.java       |  244 ++
 .../SystemCatalogCreationOnConnectionIT.java    |   34 +-
 .../apache/phoenix/end2end/SystemCatalogIT.java |   31 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |   13 +-
 .../end2end/TenantSpecificViewIndexIT.java      |   68 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   |  319 +--
 .../java/org/apache/phoenix/end2end/ViewIT.java |  868 ++++--
 .../phoenix/end2end/index/BaseIndexIT.java      |   43 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |    7 +-
 .../phoenix/end2end/index/DropColumnIT.java     |  117 -
 .../phoenix/end2end/index/IndexMetadataIT.java  |    4 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |  842 +++---
 .../phoenix/end2end/index/ViewIndexIT.java      |   68 +-
 .../apache/phoenix/execute/PartialCommitIT.java |    4 +-
 .../SystemCatalogWALEntryFilterIT.java          |   85 +-
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |    9 +-
 .../ColumnNameTrackingExpressionCompiler.java   |   46 +
 .../phoenix/compile/CreateTableCompiler.java    |    2 +-
 .../apache/phoenix/compile/FromCompiler.java    |   15 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |    2 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    2 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    2 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |    2 +-
 .../coprocessor/MetaDataEndpointImpl.java       | 2577 +++++++++---------
 .../phoenix/coprocessor/MetaDataProtocol.java   |    3 +-
 .../apache/phoenix/coprocessor/TableInfo.java   |   79 +
 .../coprocessor/TableViewFinderResult.java      |   48 +
 .../apache/phoenix/coprocessor/ViewFinder.java  |  144 +
 .../coprocessor/WhereConstantParser.java        |  106 +
 .../coprocessor/generated/MetaDataProtos.java   |  626 ++++-
 .../coprocessor/generated/PTableProtos.java     |  323 ++-
 .../phoenix/expression/LikeExpression.java      |    2 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |    8 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  534 ++--
 .../apache/phoenix/jdbc/PhoenixStatement.java   |    8 +-
 .../phoenix/parse/DropTableStatement.java       |    8 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |    2 +-
 .../phoenix/query/ConnectionQueryServices.java  |   17 +-
 .../query/ConnectionQueryServicesImpl.java      |   43 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   13 +-
 .../query/DelegateConnectionQueryServices.java  |    8 +-
 .../apache/phoenix/query/QueryConstants.java    |   14 +-
 .../org/apache/phoenix/query/QueryServices.java |    2 +
 .../phoenix/query/QueryServicesOptions.java     |    2 +
 .../SystemCatalogWALEntryFilter.java            |   45 +-
 .../apache/phoenix/schema/DelegateColumn.java   |   15 +
 .../apache/phoenix/schema/MetaDataClient.java   |   57 +-
 .../phoenix/schema/MetaDataSplitPolicy.java     |   26 +-
 .../java/org/apache/phoenix/schema/PColumn.java |   12 +
 .../org/apache/phoenix/schema/PColumnImpl.java  |  113 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    3 +-
 .../java/org/apache/phoenix/schema/PTable.java  |   17 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  279 +-
 .../org/apache/phoenix/schema/PTableKey.java    |    4 +-
 .../schema/ParentTableNotFoundException.java    |   30 +
 .../org/apache/phoenix/schema/SaltingUtil.java  |    4 +-
 .../apache/phoenix/schema/TableProperty.java    |   22 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |   16 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  171 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |    1 -
 .../org/apache/phoenix/util/SchemaUtil.java     |   43 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  186 +-
 .../phoenix/compile/QueryCompilerTest.java      |    2 +-
 .../coprocessor/MetaDataEndpointImplTest.java   |  299 ++
 .../phoenix/execute/CorrelatePlanTest.java      |    3 +-
 .../execute/LiteralResultIteratorPlanTest.java  |    4 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |    7 +-
 .../expression/ColumnExpressionTest.java        |    9 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  312 ++-
 .../apache/phoenix/util/MetaDataUtilTest.java   |   22 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   18 +-
 phoenix-protocol/src/main/MetaDataService.proto |    6 +
 phoenix-protocol/src/main/PTable.proto          |    6 +-
 pom.xml                                         |   18 +
 87 files changed, 6559 insertions(+), 3792 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
index a643383..7ef825c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
@@ -148,7 +148,7 @@ public class StatisticsCollectionRunTrackerIT extends ParallelStatsEnabledIT {
         String ddl = "CREATE TABLE " + tableName + " (PK1 VARCHAR PRIMARY KEY, KV1 VARCHAR)";
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute(ddl);
-            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_IDX ON " + tableName + "(KV1)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + generateUniqueName() + " ON " + tableName + "(KV1)");
             PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
             try (HBaseAdmin admin = phxConn.getQueryServices().getAdmin()) {
                 List<HRegionInfo> tableRegions = admin.getTableRegions(tableNameBytes);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index 7b4ff68..669b6f6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -54,7 +54,7 @@ import org.junit.Test;
 
 import com.google.common.base.Objects;
 
-public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
+public class AlterMultiTenantTableWithViewsIT extends SplitSystemCatalogIT {
 
     private Connection getTenantConnection(String tenantId) throws Exception {
         Properties tenantProps = new Properties();
@@ -84,12 +84,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testAddDropColumnToBaseTablePropagatesToEntireViewHierarchy() throws Exception {
-        String baseTable = "testViewHierarchy";
-        String baseViewName = generateUniqueName();
-        String view1 = baseViewName + "_VIEW1";
-        String view2 = baseViewName + "_VIEW2";
-        String view3 = baseViewName + "_VIEW3";
-        String view4 = baseViewName + "_VIEW4";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String view2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        String view3 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
+        String view4 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenant1 = TENANT1;
+        String tenant2 = TENANT2;
         /*                                     baseTable
                                  /                  |               \ 
                          view1(tenant1)    view3(tenant2)          view4(global)
@@ -100,7 +101,8 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
             conn.createStatement().execute(baseTableDDL);
             
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+            
+            try (Connection tenant1Conn = getTenantConnection(tenant1)) {
                 String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable;
                 tenant1Conn.createStatement().execute(view1DDL);
                 
@@ -108,7 +110,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view2DDL);
             }
             
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+            try (Connection tenant2Conn = getTenantConnection(tenant2)) {
                 String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable;
                 tenant2Conn.createStatement().execute(view3DDL);
             }
@@ -123,13 +125,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute("SELECT V3 FROM " + view4);
             
             // verify that the column is visible to view1 and view2
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+            try (Connection tenant1Conn = getTenantConnection(tenant1)) {
                 tenant1Conn.createStatement().execute("SELECT V3 from " + view1);
                 tenant1Conn.createStatement().execute("SELECT V3 from " + view2);
             }
             
             // verify that the column is visible to view3
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+            try (Connection tenant2Conn = getTenantConnection(tenant2)) {
                 tenant2Conn.createStatement().execute("SELECT V3 from " + view3);
             }
 
@@ -143,7 +145,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             } catch (ColumnNotFoundException e) {
             }
             // verify that the column is not visible to view1 and view2
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+            try (Connection tenant1Conn = getTenantConnection(tenant1)) {
                 try {
                     tenant1Conn.createStatement().execute("SELECT V1 from " + view1);
                     fail();
@@ -157,7 +159,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             }
 
             // verify that the column is not visible to view3
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+            try (Connection tenant2Conn = getTenantConnection(tenant2)) {
                 try {
                     tenant2Conn.createStatement().execute("SELECT V1 from " + view3);
                     fail();
@@ -170,12 +172,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testChangingPKOfBaseTableChangesPKForAllViews() throws Exception {
-        String baseTable = "testChangePKOfBaseTable";
-        String baseViewName = generateUniqueName();
-        String view1 = baseViewName + "_VIEW1";
-        String view2 = baseViewName + "_VIEW2";
-        String view3 = baseViewName + "_VIEW3";
-        String view4 = baseViewName + "_VIEW4";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String view2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        String view3 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
+        String view4 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenant1 = TENANT1;
+        String tenant2 = TENANT2;
         /*                                     baseTable
                                  /                  |               \ 
                          view1(tenant1)    view3(tenant2)          view4(global)
@@ -189,14 +192,14 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                     + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
             globalConn.createStatement().execute(baseTableDDL);
 
-            tenant1Conn = getTenantConnection("tenant1");
+            tenant1Conn = getTenantConnection(tenant1);
             String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable;
             tenant1Conn.createStatement().execute(view1DDL);
 
             String view2DDL = "CREATE VIEW " + view2 + " AS SELECT * FROM " + view1;
             tenant1Conn.createStatement().execute(view2DDL);
 
-            tenant2Conn = getTenantConnection("tenant2");
+            tenant2Conn = getTenantConnection(tenant2);
             String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable;
             tenant2Conn.createStatement().execute(view3DDL);
 
@@ -268,15 +271,16 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testAddPKColumnToBaseTableWhoseViewsHaveIndices() throws Exception {
-        String baseTable = "testAddPKColumnToBaseTableWhoseViewsHaveIndices";
-        String baseViewName = generateUniqueName();
-        String view1 = baseViewName + "_VIEW1";
-        String view2 = baseViewName + "_VIEW2";
-        String view3 = baseViewName + "_VIEW3";
-        String tenant1 = baseViewName + "_T1";
-        String tenant2 = baseViewName + "_T2";
-        String view2Index = view2 + "_IDX";
-        String view3Index = view3 + "_IDX";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String view2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        String view3 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
+        String view2Schema = SCHEMA3;
+        String view3Schema = SCHEMA4;
+        String tenant1 = TENANT1;
+        String tenant2 = TENANT2;
+        String view2Index = generateUniqueName() + "_IDX";
+        String view3Index = generateUniqueName() + "_IDX";
         /*                          baseTable(mutli-tenant)
                                  /                           \                
                          view1(tenant1)                  view3(tenant2, index) 
@@ -293,6 +297,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                             + " (TENANT_ID VARCHAR NOT NULL, K1 varchar not null, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, K1)) MULTI_TENANT = true ");
 
         }
+        String fullView2IndexName = SchemaUtil.getTableName(view2Schema, view2Index);
         try (Connection viewConn = getTenantConnection(tenant1)) {
             // create tenant specific view for tenant1 - view1
             viewConn.createStatement().execute("CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable);
@@ -308,9 +313,10 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
 
             // create an index on view2
             viewConn.createStatement().execute("CREATE INDEX " + view2Index + " ON " + view2 + " (v1) include (v2)");
-            assertEquals(0, getTableSequenceNumber(phxConn, view2Index));
-            assertEquals(4, getMaxKeySequenceNumber(phxConn, view2Index));
+            assertEquals(0, getTableSequenceNumber(phxConn, fullView2IndexName));
+            assertEquals(4, getMaxKeySequenceNumber(phxConn, fullView2IndexName));
         }
+        String fullView3IndexName = SchemaUtil.getTableName(view3Schema, view3Index);
         try (Connection viewConn = getTenantConnection(tenant2)) {
             // create tenant specific view for tenant2 - view3
             viewConn.createStatement().execute("CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable);
@@ -321,10 +327,8 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
 
             // create an index on view3
             viewConn.createStatement().execute("CREATE INDEX " + view3Index + " ON " + view3 + " (v1) include (v2)");
-            assertEquals(0, getTableSequenceNumber(phxConn, view3Index));
-            assertEquals(4, getMaxKeySequenceNumber(phxConn, view3Index));
-
-
+            assertEquals(0, getTableSequenceNumber(phxConn, fullView3IndexName));
+            assertEquals(4, getMaxKeySequenceNumber(phxConn, fullView3IndexName));
         }
 
         // alter the base table by adding 1 non-pk and 2 pk columns
@@ -354,7 +358,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             globalConn.commit();
         }
 
-        // Verify now that the sequence number of data table, indexes and views have changed.
+        // Verify now that the sequence number of data table, indexes and views have *not* changed.
         // Also verify that the newly added pk columns show up as pk columns of data table, indexes and views.
         try (Connection viewConn = getTenantConnection(tenant1)) {
 
@@ -362,7 +366,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
             assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view1));
             assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view1));
-            assertEquals(1, getTableSequenceNumber(phxConn, view1));
+            assertEquals(0, getTableSequenceNumber(phxConn, view1));
             assertEquals(4, getMaxKeySequenceNumber(phxConn, view1));
             verifyNewColumns(rs, "K2", "K3", "V3");
 
@@ -370,27 +374,27 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view2);
             assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view2));
             assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view2));
-            assertEquals(1, getTableSequenceNumber(phxConn, view2));
+            assertEquals(0, getTableSequenceNumber(phxConn, view2));
             assertEquals(4, getMaxKeySequenceNumber(phxConn, view2));
             verifyNewColumns(rs, "K2", "K3", "V3");
 
-            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view2Index));
-            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view2Index));
-            assertEquals(1, getTableSequenceNumber(phxConn, view2Index));
-            assertEquals(6, getMaxKeySequenceNumber(phxConn, view2Index));
+            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), fullView2IndexName));
+            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), fullView2IndexName));
+            assertEquals(0, getTableSequenceNumber(phxConn, fullView2IndexName));
+            assertEquals(6, getMaxKeySequenceNumber(phxConn, fullView2IndexName));
         }
         try (Connection viewConn = getTenantConnection(tenant2)) {
             ResultSet rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view3);
             PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
             assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view3));
             assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view3));
-            assertEquals(1, getTableSequenceNumber(phxConn, view3));
+            assertEquals(0, getTableSequenceNumber(phxConn, view3));
             verifyNewColumns(rs, "K22", "K33", "V33");
 
-            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view3Index));
-            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view3Index));
-            assertEquals(1, getTableSequenceNumber(phxConn, view3Index));
-            assertEquals(6, getMaxKeySequenceNumber(phxConn, view3Index));
+            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), fullView3IndexName));
+            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), fullView3IndexName));
+            assertEquals(0, getTableSequenceNumber(phxConn, fullView3IndexName));
+            assertEquals(6, getMaxKeySequenceNumber(phxConn, fullView3IndexName));
         }
         // Verify that the index is actually being used when using newly added pk col
         try (Connection viewConn = getTenantConnection(tenant1)) {
@@ -400,7 +404,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             Statement stmt = viewConn.createStatement();
             String sql = "SELECT V2 FROM " + view2 + " WHERE V1 = 'value1' AND K3 = 'key3'";
             QueryPlan plan = stmt.unwrap(PhoenixStatement.class).optimizeQuery(sql);
-            assertTrue(plan.getTableRef().getTable().getName().getString().equals(SchemaUtil.normalizeIdentifier(view2Index)));
+            assertEquals(fullView2IndexName, plan.getTableRef().getTable().getName().getString());
             ResultSet rs = viewConn.createStatement().executeQuery(sql);
             verifyNewColumns(rs, "value2");
         }
@@ -409,72 +413,71 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testAddingPkAndKeyValueColumnsToBaseTableWithDivergedView() throws Exception {
-        String baseTable = "testAlteringPkOfBaseTableWithDivergedView".toUpperCase();
-        String view1 = generateUniqueName();
-        String divergedView = generateUniqueName();
-        String divergedViewIndex = divergedView + "_IDX";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String divergedView = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
+        String divergedViewSchemaName = SchemaUtil.getSchemaNameFromFullName(divergedView);
+        String divergedViewIndex = generateUniqueName() + "_IDX";
+        String tenant1 = TENANT1;
+        String tenant2 = TENANT2;
+        
         /*                                     baseTable
                                  /                  |                
                          view1(tenant1)         divergedView(tenant2)    
                             
         */
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Connection tenant1Conn = getTenantConnection(tenant1);
+                Connection tenant2Conn = getTenantConnection(tenant2)) {
             String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
             conn.createStatement().execute(baseTableDDL);
             
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-            }
+            String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+            tenant1Conn.createStatement().execute(view1DDL);
             
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                String divergedViewDDL = "CREATE VIEW " + divergedView + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant2Conn.createStatement().execute(divergedViewDDL);
-                // Drop column V2 from the view to have it diverge from the base table
-                tenant2Conn.createStatement().execute("ALTER VIEW " + divergedView + " DROP COLUMN V2");
-                
-                // create an index on the diverged view
-                String indexDDL = "CREATE INDEX " + divergedViewIndex + " ON " + divergedView + " (V1) include (V3)";
-                tenant2Conn.createStatement().execute(indexDDL);
-            }
+            String divergedViewDDL = "CREATE VIEW " + divergedView + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+            tenant2Conn.createStatement().execute(divergedViewDDL);
+            // Drop column V2 from the view to have it diverge from the base table
+            tenant2Conn.createStatement().execute("ALTER VIEW " + divergedView + " DROP COLUMN V2");
+            
+            // create an index on the diverged view
+            String indexDDL = "CREATE INDEX " + divergedViewIndex + " ON " + divergedView + " (V1) include (V3)";
+            tenant2Conn.createStatement().execute(indexDDL);
             
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
             
             
             // verify that the both columns were added to view1
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                tenant1Conn.createStatement().execute("SELECT KV from " + view1);
-                tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
-            }
+            tenant1Conn.createStatement().execute("SELECT KV from " + view1);
+            tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
             
             // verify that only the primary key column PK2 was added to diverged view
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                tenant2Conn.createStatement().execute("SELECT PK2 from " + divergedView);
-                try {
-                    tenant2Conn.createStatement().execute("SELECT KV FROM " + divergedView);
-                } catch (SQLException e) {
-                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-                }
+            tenant2Conn.createStatement().execute("SELECT PK2 from " + divergedView);
+            try {
+                tenant2Conn.createStatement().execute("SELECT KV FROM " + divergedView);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
             }
             
             // Upsert records in diverged view. Verify that the PK column was added to the index on it.
             String upsert = "UPSERT INTO " + divergedView + " (PK1, PK2, V1, V3) VALUES ('PK1', 'PK2', 'V1', 'V3')";
-            try (Connection viewConn = getTenantConnection("tenant2")) {
+            try (Connection viewConn = getTenantConnection(tenant2)) {
                 viewConn.createStatement().executeUpdate(upsert);
                 viewConn.commit();
                 Statement stmt = viewConn.createStatement();
                 String sql = "SELECT V3 FROM " + divergedView + " WHERE V1 = 'V1' AND PK2 = 'PK2'";
                 QueryPlan plan = stmt.unwrap(PhoenixStatement.class).optimizeQuery(sql);
-                assertTrue(plan.getTableRef().getTable().getName().getString().equals(SchemaUtil.normalizeIdentifier(divergedViewIndex)));
+                assertEquals(SchemaUtil.getTableName(divergedViewSchemaName, divergedViewIndex),
+                    plan.getTableRef().getTable().getName().getString());
                 ResultSet rs = viewConn.createStatement().executeQuery(sql);
                 verifyNewColumns(rs, "V3");
             }
             
             // For non-diverged view, base table columns will be added at the same position as base table
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(tenant1Conn, view1, PTableType.VIEW, baseTable, 0, 7, 5,  "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
             // For a diverged view, only base table's pk column will be added and that too at the end.
-            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
+            assertTableDefinition(tenant2Conn, divergedView, PTableType.VIEW, baseTable, 1, 6, DIVERGED_VIEW_BASE_COLUMN_COUNT, "PK1", "V1", "V3", "PK2", "VIEW_COL1", "VIEW_COL2");
             
             // Adding existing column VIEW_COL2 to the base table isn't allowed.
             try {
@@ -490,83 +493,81 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testAddColumnsToSaltedBaseTableWithViews() throws Exception {
-        String baseTable = "testAddColumnsToSaltedBaseTableWithViews".toUpperCase();
-        String view1 = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenant = TENANT1;
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Connection tenant1Conn = getTenantConnection(tenant)) {
             String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
             conn.createStatement().execute(baseTableDDL);
 
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-            }
+            String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+            tenant1Conn.createStatement().execute(view1DDL);
 
             assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(tenant1Conn, view1, PTableType.VIEW, baseTable, 0, 7, 5,  "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
 
             assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(tenant1Conn, view1, PTableType.VIEW, baseTable, 0, 7, 5,  "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the both columns were added to view1
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                tenant1Conn.createStatement().execute("SELECT KV from " + view1);
-                tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
-            }
+            tenant1Conn.createStatement().execute("SELECT KV from " + view1);
+            tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
         }
     }
     
     @Test
     public void testDropColumnsFromSaltedBaseTableWithViews() throws Exception {
-        String baseTable = "testDropColumnsFromSaltedBaseTableWithViews".toUpperCase();
-        String view1 = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenant = TENANT1;
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Connection tenant1Conn = getTenantConnection(tenant)) {
             String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
             conn.createStatement().execute(baseTableDDL);
 
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-            }
+            String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+            tenant1Conn.createStatement().execute(view1DDL);
 
             assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(tenant1Conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
             conn.createStatement().execute(alterBaseTable);
 
             assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
+            // column adds and drops are no longer propagated to child views, when the parent view is resolved the dropped column is excluded
+            assertTableDefinition(tenant1Conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "PK1", "V1",  "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the dropped columns aren't visible
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                try {
-                    tenant1Conn.createStatement().execute("SELECT KV from " + view1);
-                    fail();
-                } catch (SQLException e) {
-                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-                }
-                try {
-                    tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
-                    fail();
-                } catch (SQLException e) {
-                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-                }
+            try {
+                conn.createStatement().execute("SELECT V2 from " + baseTable);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
+            }
+            try {
+                tenant1Conn.createStatement().execute("SELECT V2 from " + view1);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
             }
         }
     }
     
     @Test
     public void testAlteringViewConditionallyModifiesHTableMetadata() throws Exception {
-        String baseTable = "testAlteringViewConditionallyModifiesBaseTable".toUpperCase();
-        String view1 = generateUniqueName();
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenant = TENANT1;
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
             conn.createStatement().execute(baseTableDDL);
             HTableDescriptor tableDesc1 = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable)); 
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+            try (Connection tenant1Conn = getTenantConnection(tenant)) {
                 String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
                 tenant1Conn.createStatement().execute(view1DDL);
                 // This should not modify the base table
@@ -596,9 +597,9 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCacheInvalidatedAfterAddingColumnToBaseTableWithViews() throws Exception {
-        String baseTable = "testCacheInvalidatedAfterAddingColumnToBaseTableWithViews";
-        String viewName = baseTable + "_view";
-        String tenantId = "tenantId";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenantId = TENANT1;
         try (Connection globalConn = DriverManager.getConnection(getUrl())) {
             String tableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true" ;
             globalConn.createStatement().execute(tableDDL);
@@ -621,9 +622,9 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCacheInvalidatedAfterDroppingColumnFromBaseTableWithViews() throws Exception {
-        String baseTable = "testCacheInvalidatedAfterDroppingColumnFromBaseTableWithViews";
-        String viewName = baseTable + "_view";
-        String tenantId = "tenantId";
+        String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String tenantId = TENANT1;
         try (Connection globalConn = DriverManager.getConnection(getUrl())) {
             String tableDDL =
                     "CREATE TABLE "
@@ -656,15 +657,18 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
         }
     }
 
-    public static void assertTableDefinition(Connection conn, String tableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnName) throws Exception {
-        PreparedStatement p = conn.prepareStatement("SELECT * FROM \"SYSTEM\".\"CATALOG\" WHERE TABLE_NAME=? AND TABLE_TYPE=?");
-        p.setString(1, tableName);
-        p.setString(2, tableType.getSerializedValue());
+    public static void assertTableDefinition(Connection conn, String fullTableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnName) throws Exception {
+        String schemaName = SchemaUtil.getSchemaNameFromFullName(fullTableName);
+        String tableName = SchemaUtil.getTableNameFromFullName(fullTableName);
+        PreparedStatement p = conn.prepareStatement("SELECT * FROM \"SYSTEM\".\"CATALOG\" WHERE TABLE_SCHEM=? AND TABLE_NAME=? AND TABLE_TYPE=?");
+        p.setString(1, schemaName);
+        p.setString(2, tableName);
+        p.setString(3, tableType.getSerializedValue());
         ResultSet rs = p.executeQuery();
         assertTrue(rs.next());
-        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in BaseColumnCount"), baseColumnCount, rs.getInt("BASE_COLUMN_COUNT"));
-        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnCount"), columnCount, rs.getInt("COLUMN_COUNT"));
-        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in sequenceNumber"), sequenceNumber, rs.getInt("TABLE_SEQ_NUM"));
+        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, "Mismatch in BaseColumnCount"), baseColumnCount, rs.getInt("BASE_COLUMN_COUNT"));
+        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, "Mismatch in columnCount"), columnCount, rs.getInt("COLUMN_COUNT"));
+        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, "Mismatch in sequenceNumber"), sequenceNumber, rs.getInt("TABLE_SEQ_NUM"));
         rs.close();
     
         ResultSet parentTableColumnsRs = null; 
@@ -673,26 +677,26 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             parentTableColumnsRs.next();
         }
         
-        ResultSet viewColumnsRs = conn.getMetaData().getColumns(null, null, tableName, null);
+        ResultSet viewColumnsRs = conn.getMetaData().getColumns(null, schemaName, tableName, null);
         for (int i = 0; i < columnName.length; i++) {
             if (columnName[i] != null) {
                 assertTrue(viewColumnsRs.next());
-                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnName: i=" + i), columnName[i], viewColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME));
+                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, "Mismatch in columnName: i=" + i), columnName[i], viewColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME));
                 int viewColOrdinalPos = viewColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION);
-                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition: i=" + i), i+1, viewColOrdinalPos);
+                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, "Mismatch in ordinalPosition: i=" + i), i+1, viewColOrdinalPos);
                 // validate that all the columns in the base table are present in the view   
                 if (parentTableColumnsRs != null && !parentTableColumnsRs.isAfterLast()) {
                     ResultSetMetaData parentTableColumnsMetadata = parentTableColumnsRs.getMetaData();
                     assertEquals(parentTableColumnsMetadata.getColumnCount(), viewColumnsRs.getMetaData().getColumnCount());
                     int parentTableColOrdinalRs = parentTableColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION);
-                    assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition of view and base table for i=" + i), parentTableColOrdinalRs, viewColOrdinalPos);
+                    assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, "Mismatch in ordinalPosition of view and base table for i=" + i), parentTableColOrdinalRs, viewColOrdinalPos);
                     for (int columnIndex = 1; columnIndex < parentTableColumnsMetadata.getColumnCount(); columnIndex++) {
                         String viewColumnValue = viewColumnsRs.getString(columnIndex);
                         String parentTableColumnValue = parentTableColumnsRs.getString(columnIndex);
                         if (!Objects.equal(viewColumnValue, parentTableColumnValue)) {
                             if (parentTableColumnsMetadata.getColumnName(columnIndex).equals(PhoenixDatabaseMetaData.TABLE_NAME)) {
                                 assertEquals(parentTableName, parentTableColumnValue);
-                                assertEquals(tableName, viewColumnValue);
+                                assertEquals(fullTableName, viewColumnValue);
                             } 
                         }
                     }
@@ -700,6 +704,6 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 }
             }
         }
-        assertFalse(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, ""), viewColumnsRs.next());
+        assertFalse(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, fullTableName, ""), viewColumnsRs.next());
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index b1949ed..295b5d7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -64,6 +64,7 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -810,7 +811,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         conn1.createStatement().execute("CREATE INDEX " + indexTableName + " ON " + dataTableFullName + "(COL1) INCLUDE (COL2,COL3,COL4)");
         
         ddl = "ALTER TABLE " + dataTableFullName + " DROP COLUMN COL2, COL3";
-        conn1.createStatement().execute(ddl);
+         conn1.createStatement().execute(ddl);
         ResultSet rs = conn1.getMetaData().getColumns("", "", dataTableFullName, null);
         assertTrue(rs.next());
         assertEquals("ID",rs.getString(4));
@@ -1285,10 +1286,11 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             
             // assert that the server side metadata for the base table and the view is also updated correctly.
             assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 10));
-            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL10", schemaName, viewName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 8));
-            assertEncodedCQValue("A", "COL11", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 9);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL10", schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 8));
+            assertEncodedCQValue("A", "COL11", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 9);
             assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 4 : initBaseTableSeqNumber + 2 );
-            assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 2);
+            // view sequence number does not change as base table column changes are not propagated to views
+            assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
         }
     }
 	
@@ -1346,5 +1348,40 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         }
     }
     
+	@Test
+	public void testAlterTableWithIndexesExtendPk() throws Exception {
+		Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+		Connection conn = DriverManager.getConnection(getUrl(), props);
+		conn.setAutoCommit(false);
+		String tableName = generateUniqueName();
+		String indexName1 = "I_" + generateUniqueName();
+		String indexName2 = "I_" + generateUniqueName();
+
+		try {
+			String ddl = "CREATE TABLE " + tableName + " (ORG_ID CHAR(15) NOT NULL,"
+					+ " PARTITION_KEY CHAR(3) NOT NULL, " + " ACTIVITY_DATE DATE NOT NULL, "
+					+ " FK1_ID CHAR(15) NOT NULL, " + " FK2_ID CHAR(15) NOT NULL, " + " TYPE VARCHAR NOT NULL, "
+					+ " IS_OPEN BOOLEAN " + " CONSTRAINT PKVIEW PRIMARY KEY " + "("
+					+ "ORG_ID, PARTITION_KEY, ACTIVITY_DATE, FK1_ID, FK2_ID, TYPE" + "))";
+			createTestTable(getUrl(), ddl);
+
+			String idx1ddl = "CREATE INDEX " + indexName1 + " ON " + tableName
+					+ " (FK1_ID, ACTIVITY_DATE DESC) INCLUDE (IS_OPEN)";
+			PreparedStatement stmt1 = conn.prepareStatement(idx1ddl);
+			stmt1.execute();
+
+			String idx2ddl = "CREATE INDEX " + indexName2 + " ON " + tableName
+					+ " (FK2_ID, ACTIVITY_DATE DESC) INCLUDE (IS_OPEN)";
+			PreparedStatement stmt2 = conn.prepareStatement(idx2ddl);
+			stmt2.execute();
+
+			ddl = "ALTER TABLE " + tableName + " ADD SOURCE VARCHAR(25) NULL PRIMARY KEY";
+			PreparedStatement stmt3 = conn.prepareStatement(ddl);
+			stmt3.execute();
+		} finally {
+			conn.close();
+		}
+	}
+
 }
  


[33/50] [abbrv] phoenix git commit: PHOENIX-3236 Problem with shading apache commons on Azure.

Posted by ja...@apache.org.
PHOENIX-3236 Problem with shading apache commons on Azure.


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

Branch: refs/heads/omid2
Commit: 79e8337500b171f02b8740c0a809a2ffb9d0d1cd
Parents: a575ac0
Author: Sergey Soldatov <ss...@apache.org>
Authored: Thu Sep 1 00:08:14 2016 -0700
Committer: Sergey Soldatov <ss...@apache.org>
Committed: Fri Aug 24 10:57:37 2018 -0700

----------------------------------------------------------------------
 phoenix-client/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/79e83375/phoenix-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index c8fb6aa..b3cc9e5 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -275,6 +275,7 @@
                   <excludes>
                     <exclude>org.apache.commons.csv.**</exclude>
                     <exclude>org.apache.commons.logging.**</exclude>
+                    <exclude>org.apache.commons.configuration.**</exclude>
                   </excludes>
                 </relocation>
                 <relocation>


[14/50] [abbrv] phoenix git commit: PHOENIX-4817 Fixed Phoenix Tracing Web Application (fixed check null, trace table name, webapp path, column names) and traceserver.py

Posted by ja...@apache.org.
PHOENIX-4817 Fixed Phoenix Tracing Web Application (fixed check null, trace table name, webapp path, column names) and traceserver.py

Closes #311

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: 1f7e3206bc0b1039b4b8a5fa9a2c8d06eb64e9d3
Parents: 1301420
Author: Vitaly Monastyrev <VM...@domain.corp>
Authored: Tue Jul 24 12:14:12 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jul 25 14:29:27 2018 -0400

----------------------------------------------------------------------
 bin/traceserver.py                              |  6 +-
 .../apache/phoenix/tracingwebapp/http/Main.java | 13 +--
 .../tracingwebapp/http/TraceServlet.java        | 87 +++++++++++---------
 3 files changed, 62 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1f7e3206/bin/traceserver.py
----------------------------------------------------------------------
diff --git a/bin/traceserver.py b/bin/traceserver.py
index 665099e..62e168c 100755
--- a/bin/traceserver.py
+++ b/bin/traceserver.py
@@ -116,8 +116,10 @@ else:
 
 #    " -Xdebug -Xrunjdwp:transport=dt_socket,address=5005,server=y,suspend=n " + \
 #    " -XX:+UnlockCommercialFeatures -XX:+FlightRecorder -XX:FlightRecorderOptions=defaultrecording=true,dumponexit=true" + \
-java_cmd = '%(java)s $PHOENIX_OPTS ' + \
-    '-cp ' + hbase_config_path + os.pathsep + phoenix_utils.phoenix_traceserver_jar + os.pathsep + phoenix_utils.phoenix_client_jar + \
+java_cmd = '%(java)s  ' + \
+    '-cp ' + hbase_config_path + os.pathsep + phoenix_utils.phoenix_traceserver_jar + os.pathsep + \
+    phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.phoenix_queryserver_jar + \
+    os.pathsep + phoenix_utils.hadoop_classpath + \
     " -Dproc_phoenixtraceserver" + \
     " -Dlog4j.configuration=file:" + os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " -Dpsql.root.logger=%(root_logger)s" + \

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1f7e3206/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/Main.java
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/Main.java b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/Main.java
index 5875fc1..249f8e6 100755
--- a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/Main.java
+++ b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/Main.java
@@ -47,6 +47,7 @@ public final class Main extends Configured implements Tool {
     public static final String TRACE_SERVER_HTTP_JETTY_HOME_KEY =
             "phoenix.traceserver.http.home";
     public static final String DEFAULT_HTTP_HOME = "/";
+    public static final String DEFAULT_WEBAPP_DIR_LOCATION = "src/main/webapp";
 
     public static void main(String[] args) throws Exception {
         int ret = ToolRunner.run(HBaseConfiguration.create(), new Main(), args);
@@ -62,15 +63,17 @@ public final class Main extends Configured implements Tool {
         final String home = getConf().get(TRACE_SERVER_HTTP_JETTY_HOME_KEY,
                 DEFAULT_HTTP_HOME);
         //setting up the embedded server
-        ProtectionDomain domain = Main.class.getProtectionDomain();
-        URL location = domain.getCodeSource().getLocation();
-        String webappDirLocation = location.toString().split("target")[0] +"src/main/webapp";
         Server server = new Server(port);
         WebAppContext root = new WebAppContext();
 
+        URL webAppDir = Thread.currentThread().getContextClassLoader().getResource(DEFAULT_WEBAPP_DIR_LOCATION);
+        if (webAppDir == null) {
+            throw new RuntimeException(String.format("No %s directory was found into the JAR file", DEFAULT_WEBAPP_DIR_LOCATION));
+        }
+
         root.setContextPath(home);
-        root.setDescriptor(webappDirLocation + "/WEB-INF/web.xml");
-        root.setResourceBase(webappDirLocation);
+        root.setDescriptor(DEFAULT_WEBAPP_DIR_LOCATION + "/WEB-INF/web.xml");
+        root.setResourceBase(webAppDir.toURI().toString());
         root.setParentLoaderPriority(true);
         server.setHandler(root);
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1f7e3206/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
index c20b20d..98250e8 100755
--- a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
+++ b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
@@ -24,8 +24,11 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.codehaus.jackson.map.ObjectMapper;
-import org.apache.phoenix.metrics.MetricInfo;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.List;
@@ -42,14 +45,21 @@ public class TraceServlet extends HttpServlet {
   private static Connection con;
   protected String DEFAULT_LIMIT = "25";
   protected String DEFAULT_COUNTBY = "hostname";
+  protected String DESCRIPTION_COUNTBY = "description";
   protected String LOGIC_AND = "AND";
   protected String LOGIC_OR = "OR";
   protected String TRACING_TABLE = "SYSTEM.TRACING_STATS";
 
-
+  @Override
+  public void init() {
+    Configuration conf = HBaseConfiguration.create();
+    TRACING_TABLE =
+            conf.get(QueryServices.TRACING_STATS_TABLE_NAME_ATTRIB,
+                    QueryServicesOptions.DEFAULT_TRACING_STATS_TABLE_NAME);
+  }
 
   protected void doGet(HttpServletRequest request, HttpServletResponse response)
-      throws ServletException, IOException {
+          throws ServletException, IOException {
 
     //reading url params
     String action = request.getParameter("action");
@@ -60,7 +70,7 @@ public class TraceServlet extends HttpServlet {
     if ("getall".equals(action)) {
       jsonObject = getAll(limit);
     } else if ("getCount".equals(action)) {
-      jsonObject = getCount("description");
+      jsonObject = getCount(DESCRIPTION_COUNTBY);
     } else if ("getDistribution".equals(action)) {
       jsonObject = getCount(DEFAULT_COUNTBY);
     } else if ("searchTrace".equals(action)) {
@@ -98,33 +108,36 @@ public class TraceServlet extends HttpServlet {
     if(countby == null) {
       countby = DEFAULT_COUNTBY;
     }
-    // Throws exception if the column not present in the trace table.
-    MetricInfo.getColumnName(countby.toLowerCase());
     String sqlQuery = "SELECT "+countby+", COUNT(*) AS count FROM " + TRACING_TABLE + " GROUP BY "+countby+" HAVING COUNT(*) > 1 ";
     json = getResults(sqlQuery);
     return json;
   }
 
   //search the trace over parent id or trace id
-  protected String searchTrace(String parentId, String traceId,String logic) {
+  protected String searchTrace(String parentId, String traceId, String logic) {
+
     String json = null;
     String query = null;
     // Check the parent Id, trace id type or long or not.
     try {
+      if (parentId != null) {
         Long.parseLong(parentId);
+      }
+      if (traceId != null) {
         Long.parseLong(traceId);
+      }
     } catch (NumberFormatException e) {
-    	throw new RuntimeException("The passed parentId/traceId is not a number.", e);
+      throw new RuntimeException("The passed parentId/traceId is not a number.", e);
     }
-    if(!logic.equals(LOGIC_AND) || !logic.equals(LOGIC_OR)) {
-    	throw new RuntimeException("Wrong logical operator passed to the query. Only "+ LOGIC_AND+","+LOGIC_OR+" are allowed.") ;
+    if (logic != null && !logic.equals(LOGIC_AND) && !logic.equals(LOGIC_OR)) {
+      throw new RuntimeException("Wrong logical operator passed to the query. Only " + LOGIC_AND + "," + LOGIC_OR + " are allowed.");
     }
-    if(parentId != null && traceId != null) {
-      query = "SELECT * FROM " + TRACING_TABLE + " WHERE parent_id="+parentId+" "+logic+" trace_id="+traceId;
-    }else if (parentId != null && traceId == null) {
-      query = "SELECT * FROM " + TRACING_TABLE + " WHERE parent_id="+parentId;
-    }else if(parentId == null && traceId != null) {
-      query = "SELECT * FROM " + TRACING_TABLE + " WHERE trace_id="+traceId;
+    if (parentId != null && traceId != null) {
+      query = "SELECT * FROM " + TRACING_TABLE + " WHERE parent_id=" + parentId + " " + logic + " trace_id=" + traceId;
+    } else if (parentId != null && traceId == null) {
+      query = "SELECT * FROM " + TRACING_TABLE + " WHERE parent_id=" + parentId;
+    } else if (parentId == null && traceId != null) {
+      query = "SELECT * FROM " + TRACING_TABLE + " WHERE trace_id=" + traceId;
     }
     json = getResults(query);
     return getJson(json);
@@ -133,37 +146,37 @@ public class TraceServlet extends HttpServlet {
   //return json string
   protected String getJson(String json) {
     String output = json.toString().replace("_id\":", "_id\":\"")
-        .replace(",\"hostname", "\",\"hostname")
-        .replace(",\"parent", "\",\"parent")
-        .replace(",\"end", "\",\"end");
+            .replace(",\"hostname", "\",\"hostname")
+            .replace(",\"parent", "\",\"parent")
+            .replace(",\"end", "\",\"end");
     return output;
   }
 
   //get results with passing sql query
   protected String getResults(String sqlQuery) {
     String json = null;
-    if(sqlQuery == null){
+    if (sqlQuery == null) {
       json = "{error:true,msg:'SQL was null'}";
-    }else{
-    try {
-      con = ConnectionFactory.getConnection();
-      EntityFactory nutrientEntityFactory = new EntityFactory(con,sqlQuery);
-      List<Map<String, Object>> nutrients = nutrientEntityFactory
-          .findMultiple();
-      ObjectMapper mapper = new ObjectMapper();
-      json = mapper.writeValueAsString(nutrients);
-    } catch (Exception e) {
-      json = "{error:true,msg:'Serrver Error:"+e.getMessage()+"'}";
-    } finally {
-      if (con != null) {
-        try {
-          con.close();
-        } catch (SQLException e) {
-          json = "{error:true,msg:'SQL Serrver Error:"+e.getMessage()+"'}";
+    } else {
+      try {
+        con = ConnectionFactory.getConnection();
+        EntityFactory nutrientEntityFactory = new EntityFactory(con, sqlQuery);
+        List<Map<String, Object>> nutrients = nutrientEntityFactory
+                .findMultiple();
+        ObjectMapper mapper = new ObjectMapper();
+        json = mapper.writeValueAsString(nutrients);
+      } catch (Exception e) {
+        json = "{error:true,msg:'Server Error:" + e.getMessage() + "'}";
+      } finally {
+        if (con != null) {
+          try {
+            con.close();
+          } catch (SQLException e) {
+            json = "{error:true,msg:'SQL Serrver Error:" + e.getMessage() + "'}";
+          }
         }
       }
     }
-    }
     return json;
   }
 }


[21/50] [abbrv] phoenix git commit: PHOENIX-4799 Write cells using checkAndMutate to prevent conflicting changes

Posted by ja...@apache.org.
PHOENIX-4799 Write cells using checkAndMutate to prevent conflicting changes


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

Branch: refs/heads/omid2
Commit: 7a2531d5a9efbefbcf158f73cbdc218d3556354c
Parents: 17773a8
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Jul 24 10:49:31 2018 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Mon Aug 6 10:53:18 2018 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   2 +-
 .../phoenix/end2end/BasePermissionsIT.java      |  66 +++---
 .../phoenix/end2end/ChangePermissionsIT.java    |   4 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |  13 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |   4 +
 .../phoenix/end2end/TableDDLPermissionsIT.java  |   9 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |   2 +
 .../org/apache/phoenix/end2end/UpgradeIT.java   |  34 +--
 .../java/org/apache/phoenix/end2end/ViewIT.java | 235 +++++++++++++++++--
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   2 +
 .../phoenix/query/ConnectionQueryServices.java  |  14 ++
 .../query/ConnectionQueryServicesImpl.java      | 170 +++++++-------
 .../query/ConnectionlessQueryServicesImpl.java  |  20 ++
 .../query/DelegateConnectionQueryServices.java  |  11 +
 .../apache/phoenix/query/QueryConstants.java    |  14 ++
 .../apache/phoenix/schema/MetaDataClient.java   |  95 +++++++-
 16 files changed, 523 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 e39d492..e97a40d 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
@@ -530,7 +530,7 @@ public class AlterTableWithViewsIT extends SplitSystemCatalogIT {
             
             try {
                 // should fail because there are two view with different pk columns
-                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
index d33d538..88a942e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
@@ -16,8 +16,29 @@
  */
 package org.apache.phoenix.end2end;
 
-import com.google.common.base.Joiner;
-import com.google.common.base.Throwables;
+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.lang.reflect.UndeclaredThrowableException;
+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.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,34 +57,13 @@ import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.QueryUtil;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
-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.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-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 com.google.common.base.Joiner;
+import com.google.common.base.Throwables;
 
 @RunWith(Parameterized.class)
 public class BasePermissionsIT extends BaseTest {
@@ -73,17 +73,23 @@ public class BasePermissionsIT extends BaseTest {
     static String SUPERUSER;
 
     static HBaseTestingUtility testUtil;
-    static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
-            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION"));
+    static final Set<String> PHOENIX_SYSTEM_TABLES =
+            new HashSet<>(Arrays.asList("SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS",
+                "SYSTEM.FUNCTION", "SYSTEM.MUTEX"));
 
-    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS = new HashSet<>(Arrays.asList(
-            "SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\""));
+    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS =
+            new HashSet<>(Arrays.asList("SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"",
+                "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\"", "SYSTEM.\"MUTEX\""));
 
     static final String SYSTEM_SEQUENCE_IDENTIFIER =
             QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
 
+    static final String SYSTEM_MUTEX_IDENTIFIER =
+            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\""
+                    + PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME + "\"";
+
     static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
-            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION"));
+            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION", "SYSTEM:MUTEX"));
 
     // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
     // Permissions can be granted or revoke by superusers and admins only

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
index 3965f69..7f2964d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
@@ -57,7 +57,8 @@ public class ChangePermissionsIT extends BasePermissionsIT {
             } else {
                 verifyAllowed(grantPermissions("RX", user, PHOENIX_SYSTEM_TABLES_IDENTIFIERS, false), superUser);
             }
-            verifyAllowed(grantPermissions("W", user, SYSTEM_SEQUENCE_IDENTIFIER, false), superUser);
+            verifyAllowed(grantPermissions("RWX", user, SYSTEM_SEQUENCE_IDENTIFIER, false), superUser);
+            verifyAllowed(grantPermissions("RWX", user, SYSTEM_MUTEX_IDENTIFIER, false), superUser);
         }
     }
 
@@ -69,6 +70,7 @@ public class ChangePermissionsIT extends BasePermissionsIT {
                 verifyAllowed(revokePermissions(user, PHOENIX_SYSTEM_TABLES_IDENTIFIERS, false), superUser);
             }
             verifyAllowed(revokePermissions(user, SYSTEM_SEQUENCE_IDENTIFIER, false), superUser);
+            verifyAllowed(revokePermissions(user, SYSTEM_MUTEX_IDENTIFIER, false), superUser);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 d253f6e..ffac4d6 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
@@ -54,7 +54,6 @@ import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -275,16 +274,15 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
 
     private void changeMutexLock(Properties clientProps, boolean acquire) throws SQLException, IOException {
         ConnectionQueryServices services = null;
-        byte[] mutexRowKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
-                PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE);
 
         try (Connection conn = DriverManager.getConnection(getJdbcUrl(), clientProps)) {
             services = conn.unwrap(PhoenixConnection.class).getQueryServices();
             if(acquire) {
                assertTrue(((ConnectionQueryServicesImpl) services)
-                        .acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP, mutexRowKey));
+                        .acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP));
             } else {
-                ((ConnectionQueryServicesImpl) services).releaseUpgradeMutex(mutexRowKey);
+                services.deleteMutexCell(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
+                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, null, null);
             }
         }
     }
@@ -397,14 +395,13 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
             }
         }
 
-        // 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);
+            assertTrue(namespaceMappedSystemTablesSet.isEmpty());
         } else {
-            assertTrue(systemTablesSet.size() == 1);
+            assertTrue(systemTablesSet.isEmpty());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 90f9db6..7d31c62 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
@@ -21,6 +21,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCH
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.CUSTOM_ENTITY_DATA_FULL_NAME;
@@ -175,6 +176,9 @@ 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(SYSTEM_MUTEX_TABLE_NAME, rs.getString("TABLE_NAME"));
+            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/7a2531d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
index 8666bb8..86a6b60 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
@@ -20,7 +20,6 @@ import java.security.PrivilegedExceptionAction;
 import java.sql.Connection;
 import java.util.Collections;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
@@ -54,6 +53,10 @@ public class TableDDLPermissionsIT extends BasePermissionsIT {
                         Action.READ, Action.EXEC);
                 grantPermissions(unprivilegedUser.getShortName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
                         Action.READ, Action.EXEC);
+                grantPermissions(regularUser1.getShortName(), Collections.singleton("SYSTEM:MUTEX"), Action.WRITE,
+                    Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortName(), Collections.singleton("SYSTEM:MUTEX"), Action.WRITE,
+                    Action.READ, Action.EXEC);
                 
             } else {
                 grantPermissions(regularUser1.getShortName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
@@ -64,6 +67,10 @@ public class TableDDLPermissionsIT extends BasePermissionsIT {
                         Action.READ, Action.EXEC);
                 grantPermissions(unprivilegedUser.getShortName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
                         Action.READ, Action.EXEC);
+                grantPermissions(regularUser1.getShortName(), Collections.singleton("SYSTEM.MUTEX"), Action.WRITE,
+                    Action.READ, Action.EXEC);
+            grantPermissions(unprivilegedUser.getShortName(), Collections.singleton("SYSTEM.MUTEX"), Action.WRITE,
+                    Action.READ, Action.EXEC);
             }
         } catch (Throwable e) {
             if (e instanceof Exception) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 dd6f7f7..956b43c 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
@@ -498,6 +498,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             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.SYSTEM_MUTEX_TABLE_NAME, 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/7a2531d5/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 2b866a5..c2cf9e6 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
@@ -18,8 +18,6 @@
 package org.apache.phoenix.end2end;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.phoenix.query.ConnectionQueryServicesImpl.UPGRADE_MUTEX;
-import static org.apache.phoenix.query.ConnectionQueryServicesImpl.UPGRADE_MUTEX_UNLOCKED;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -42,8 +40,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.curator.shaded.com.google.common.collect.Sets;
 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.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -425,39 +421,21 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         }
     }
     
-    private void putUnlockKVInSysMutex(byte[] row) throws Exception {
-        try (Connection conn = getConnection(false, null)) {
-            ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-            try (HTableInterface sysMutexTable = services.getTable(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME_BYTES)) {
-                byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
-                byte[] qualifier = UPGRADE_MUTEX;
-                Put put = new Put(row);
-                put.addColumn(family, qualifier, UPGRADE_MUTEX_UNLOCKED);
-                sysMutexTable.put(put);
-                sysMutexTable.flushCommits();
-            }
-        }
-    }
-    
     @Test
     public void testAcquiringAndReleasingUpgradeMutex() throws Exception {
         ConnectionQueryServices services = null;
-        byte[] mutexRowKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
-                generateUniqueName());
         try (Connection conn = getConnection(false, null)) {
             services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-            putUnlockKVInSysMutex(mutexRowKey);
             assertTrue(((ConnectionQueryServicesImpl)services)
-                    .acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0, mutexRowKey));
+                    .acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0));
             try {
                 ((ConnectionQueryServicesImpl)services)
-                        .acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0, mutexRowKey);
+                        .acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                 fail();
             } catch (UpgradeInProgressException expected) {
 
             }
-            assertTrue(((ConnectionQueryServicesImpl)services).releaseUpgradeMutex(mutexRowKey));
-            assertFalse(((ConnectionQueryServicesImpl)services).releaseUpgradeMutex(mutexRowKey));
+            ((ConnectionQueryServicesImpl)services).releaseUpgradeMutex();
         }
     }
     
@@ -471,7 +449,6 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         final byte[] mutexKey = Bytes.toBytes(generateUniqueName());
         try (Connection conn = getConnection(false, null)) {
             services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-            putUnlockKVInSysMutex(mutexKey);
             FutureTask<Void> task1 = new FutureTask<>(new AcquireMutexRunnable(mutexStatus1, services, latch, numExceptions, mutexKey));
             FutureTask<Void> task2 = new FutureTask<>(new AcquireMutexRunnable(mutexStatus2, services, latch, numExceptions, mutexKey));
             Thread t1 = new Thread(task1);
@@ -509,12 +486,15 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         public Void call() throws Exception {
             try {
                 ((ConnectionQueryServicesImpl)services).acquireUpgradeMutex(
-                        MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0, mutexRowKey);
+                        MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                 acquireStatus.set(true);
             } catch (UpgradeInProgressException e) {
                 numExceptions.incrementAndGet();
             } finally {
                 latch.countDown();
+                if (acquireStatus.get()) {
+                    ((ConnectionQueryServicesImpl)services).releaseUpgradeMutex();
+                }
             }
             return null;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index fdfd75b..fda9490 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -20,11 +20,13 @@ package org.apache.phoenix.end2end;
 import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
 import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MODIFY_VIEW_PK;
 import static org.apache.phoenix.exception.SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.analyzeTable;
 import static org.apache.phoenix.util.TestUtil.getAllSplits;
 import static org.junit.Assert.assertArrayEquals;
 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;
 
@@ -43,6 +45,13 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.curator.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -52,9 +61,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -67,6 +76,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
+import org.apache.phoenix.schema.ConcurrentTableMutationException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
@@ -74,6 +84,7 @@ import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.MetaDataUtil;
 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.SchemaUtil;
@@ -93,6 +104,17 @@ public class ViewIT extends SplitSystemCatalogIT {
 
     protected String tableDDLOptions;
     protected boolean transactional;
+    
+    private static final String FAILED_VIEWNAME = "FAILED_VIEW";
+    private static final byte[] FAILED_ROWKEY_BYTES =
+            SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2), Bytes.toBytes(FAILED_VIEWNAME));
+    private static final String SLOW_VIEWNAME_PREFIX = "SLOW_VIEW";
+    private static final byte[] SLOW_ROWKEY_PREFIX_BYTES =
+            SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2),
+                Bytes.toBytes(SLOW_VIEWNAME_PREFIX));
+
+    private static volatile CountDownLatch latch1 = null;
+    private static volatile CountDownLatch latch2 = null;
 
     public ViewIT(boolean transactional) {
         StringBuilder optionBuilder = new StringBuilder();
@@ -114,7 +136,7 @@ public class ViewIT extends SplitSystemCatalogIT {
         Map<String, String> props = Collections.emptyMap();
         boolean splitSystemCatalog = (driver == null);
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
-        serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+        serverProps.put("hbase.coprocessor.region.classes", TestMetaDataRegionObserver.class.getName());
         serverProps.put("hbase.coprocessor.abortonerror", "false");
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(props.entrySet().iterator()));
         // Split SYSTEM.CATALOG once after the mini-cluster is started
@@ -123,6 +145,49 @@ public class ViewIT extends SplitSystemCatalogIT {
         }
     }
     
+    public static class TestMetaDataRegionObserver extends BaseRegionObserver {
+        @Override
+        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+                MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+            if (shouldFail(c, miniBatchOp.getOperation(0))) {
+                // throwing anything other than instances of IOException result
+                // in this coprocessor being unloaded
+                // DoNotRetryIOException tells HBase not to retry this mutation
+                // multiple times
+                throw new DoNotRetryIOException();
+            } else if (shouldSlowDown(c, miniBatchOp.getOperation(0))) {
+                // simulate a slow write to SYSTEM.CATALOG
+                if (latch1 != null) {
+                    latch1.countDown();
+                }
+                if (latch2 != null) {
+                    try {
+                        // wait till the second task is complete before completing the first task
+                        boolean result = latch2.await(2, TimeUnit.MINUTES);
+                        if (!result) {
+                            throw new RuntimeException("Second task took took long to complete");
+                        }
+                    } catch (InterruptedException e) {
+                    }
+                }
+            }
+        }
+
+        private boolean shouldFail(ObserverContext<RegionCoprocessorEnvironment> c, Mutation m) {
+            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
+            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
+                    && (Bytes.equals(FAILED_ROWKEY_BYTES, m.getRow()));
+        }
+
+        private boolean shouldSlowDown(ObserverContext<RegionCoprocessorEnvironment> c,
+                Mutation m) {
+            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
+            byte[] rowKeyPrefix = Arrays.copyOf(m.getRow(), SLOW_ROWKEY_PREFIX_BYTES.length);
+            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
+                    && (Bytes.equals(SLOW_ROWKEY_PREFIX_BYTES, rowKeyPrefix));
+        }
+    }
+    
     @Test
     public void testReadOnlyOnUpdatableView() throws Exception {
         String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
@@ -1274,28 +1339,160 @@ public class ViewIT extends SplitSystemCatalogIT {
         PhoenixRuntime.getTableNoCache(conn, fullViewName2);
     }
     
-    private static final String FAILED_VIEWNAME = "FAILED_VIEW";
-    private static final byte[] ROWKEY_TO_FAIL_BYTES = SchemaUtil.getTableKey(null, Bytes.toBytes(SCHEMA2),
-            Bytes.toBytes(FAILED_VIEWNAME));
-    
-    public static class FailingRegionObserver extends SimpleRegionObserver {
+    @Test
+    public void testConcurrentViewCreationAndTableDrop() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String fullViewName1 =
+                    SchemaUtil.getTableName(SCHEMA2,
+                        SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
+            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            String tableDdl =
+                    "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                            + tableDDLOptions;
+            conn.createStatement().execute(tableDdl);
+
+            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // create the view in a separate thread (which will take some time
+            // to complete)
+            Future<Exception> future =
+                    executorService.submit(new CreateViewRunnable(fullTableName, fullViewName1));
+            // wait till the thread makes the rpc to create the view
+            latch1.await();
+            tableDdl = "DROP TABLE " + fullTableName;
+            try {
+                // drop table should fail as we are concurrently adding a view
+                conn.createStatement().execute(tableDdl);
+                fail("Creating a view while concurrently dropping the base table should fail");
+            } catch (ConcurrentTableMutationException e) {
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertNull(e);
+
+            // create another view to ensure that the cell used to prevent
+            // concurrent modifications was removed
+            String ddl =
+                    "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM "
+                            + fullTableName + " WHERE k = 6";
+            conn.createStatement().execute(ddl);
+        }
+    }
+
+    @Test
+    public void testConcurrentAddColumn() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String fullViewName1 =
+                    SchemaUtil.getTableName(SCHEMA2,
+                        SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName());
+            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            // create base table
+            String tableDdl =
+                    "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                            + tableDDLOptions;
+            conn.createStatement().execute(tableDdl);
+            // create a view
+            String ddl =
+                    "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
+                            + fullTableName + " WHERE k = 6";
+            conn.createStatement().execute(ddl);
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // add a column to the view in a separate thread (which will take
+            // some time to complete)
+            Future<Exception> future = executorService.submit(new AddColumnRunnable(fullViewName1));
+            // wait till the thread makes the rpc to create the view
+            boolean result = latch1.await(2, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The create view rpc look too long");
+            }
+            tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
+            try {
+                // add the same column to the base table with a different type
+                conn.createStatement().execute(tableDdl);
+                fail("Creating a view while concurrently dropping the base table should fail");
+            } catch (ConcurrentTableMutationException e) {
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertNull(e);
+
+            // add a the same column to the another view  to ensure that the cell used
+            // to prevent concurrent modifications was removed
+            ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " 
+                    + fullTableName + " WHERE k = 6";
+            conn.createStatement().execute(ddl);
+            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
+            conn.createStatement().execute(tableDdl);
+        }
+    }
+
+    private class CreateViewRunnable implements Callable<Exception> {
+        private final String fullTableName;
+        private final String fullViewName;
+
+        public CreateViewRunnable(String fullTableName, String fullViewName) {
+            this.fullTableName = fullTableName;
+            this.fullViewName = fullViewName;
+        }
+
         @Override
-        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
-                MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
-            if (shouldFail(c, miniBatchOp.getOperation(0))) {
-                // throwing anything other than instances of IOException result
-                // in this coprocessor being unloaded
-                // DoNotRetryIOException tells HBase not to retry this mutation
-                // multiple times
-                throw new DoNotRetryIOException();
+        public Exception call() throws Exception {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                String ddl =
+                        "CREATE VIEW " + fullViewName + " (v2 VARCHAR) AS SELECT * FROM "
+                                + fullTableName + " WHERE k = 5";
+                conn.createStatement().execute(ddl);
+            } catch (SQLException e) {
+                return e;
             }
+            return null;
         }
+    }
 
-        private boolean shouldFail(ObserverContext<RegionCoprocessorEnvironment> c, Mutation m) {
-            TableName tableName = c.getEnvironment().getRegion().getRegionInfo().getTable();
-            return tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME)
-                    && (Bytes.equals(ROWKEY_TO_FAIL_BYTES, m.getRow()));
+    private class AddColumnRunnable implements Callable<Exception> {
+        private final String fullViewName;
+
+        public AddColumnRunnable(String fullViewName) {
+            this.fullViewName = fullViewName;
         }
 
+        @Override
+        public Exception call() throws Exception {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                String ddl = "ALTER VIEW " + fullViewName + " ADD v3 CHAR(15)";
+                conn.createStatement().execute(ddl);
+            } catch (SQLException e) {
+                return e;
+            }
+            return null;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 dab1048..7a4a481 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
@@ -331,6 +331,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String SEQUENCE_TABLE_TYPE = SYSTEM_SEQUENCE_TABLE;
 
     public static final String SYNC_INDEX_CREATED_DATE = "SYNC_INDEX_CREATED_DATE";
+    public static final String SYSTEM_MUTEX_COLUMN_NAME = "MUTEX_VALUE";
+    public static final byte[] SYSTEM_MUTEX_COLUMN_NAME_BYTES = Bytes.toBytes(SYSTEM_MUTEX_COLUMN_NAME);
     public static final String SYSTEM_MUTEX_TABLE_NAME = "MUTEX";
     public static final String SYSTEM_MUTEX_NAME = SchemaUtil.getTableName(QueryConstants.SYSTEM_SCHEMA_NAME, SYSTEM_MUTEX_TABLE_NAME);
     public static final TableName SYSTEM_MUTEX_HBASE_TABLE_NAME = TableName.valueOf(SYSTEM_MUTEX_NAME);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 6f8cbc0..0820232 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
@@ -169,4 +169,18 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public QueryLoggerDisruptor getQueryDisruptor();
     
     public PhoenixTransactionClient initTransactionClient(TransactionFactory.Provider provider);
+    
+    /**
+     * Writes a cell to SYSTEM.MUTEX using checkAndPut to ensure only a single client can execute a
+     * particular task. The params are used to generate the rowkey.
+     * @return true if this client was able to successfully acquire the mutex
+     */
+    public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException;
+
+    /**
+     * Deletes a cell that was written to SYSTEM.MUTEX. The params are used to generate the rowkey.
+     */
+    public void deleteMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 4c7630d..8bbb379 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
@@ -64,6 +64,7 @@ import static org.apache.phoenix.util.UpgradeUtil.moveChildLinks;
 import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
 
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
 import java.lang.ref.WeakReference;
 import java.sql.PreparedStatement;
 import java.sql.ResultSetMetaData;
@@ -109,6 +110,7 @@ import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.TableExistsException;
 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.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -326,9 +328,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private final AtomicBoolean upgradeRequired = new AtomicBoolean(false);
     private final int maxConnectionsAllowed;
     private final boolean shouldThrottleNumConnections;
-    public static final byte[] UPGRADE_MUTEX = "UPGRADE_MUTEX".getBytes();
-    public static final byte[] UPGRADE_MUTEX_LOCKED = "UPGRADE_MUTEX_LOCKED".getBytes();
-    public static final byte[] UPGRADE_MUTEX_UNLOCKED = "UPGRADE_MUTEX_UNLOCKED".getBytes();
+    public static final byte[] MUTEX_LOCKED = "MUTEX_LOCKED".getBytes();
 
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
@@ -2502,6 +2502,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     protected String getChildLinkDDL() {
         return setSystemDDLProperties(QueryConstants.CREATE_CHILD_LINK_METADATA);
     }
+    
+    protected String getMutexDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_MUTEX_METADTA);
+    }
 
     private String setSystemDDLProperties(String ddl) {
         return String.format(ddl,
@@ -2673,13 +2677,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             columnDesc.setTimeToLive(TTL_FOR_MUTEX); // Let mutex expire after some time
             tableDesc.addFamily(columnDesc);
             admin.createTable(tableDesc);
-            try (HTableInterface sysMutexTable = getTable(mutexTableName.getName())) {
-                byte[] mutexRowKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
-                        PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE);
-                Put put = new Put(mutexRowKey);
-                put.addColumn(PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES, UPGRADE_MUTEX, UPGRADE_MUTEX_UNLOCKED);
-                sysMutexTable.put(put);
-            }
         } catch (IOException e) {
             if(!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), AccessDeniedException.class)) ||
                     !Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), org.apache.hadoop.hbase.TableNotFoundException.class))) {
@@ -2687,13 +2684,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } else {
                 throw e;
             }
-        }catch(PhoenixIOException e){
-            if(e.getCause()!=null && e.getCause() instanceof AccessDeniedException)
-            {
-                //Ignore
-            }else{
-                throw e;
-            }
         }
     }
 
@@ -2723,13 +2713,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.createStatement().executeUpdate(getChildLinkDDL());
         } catch (TableAlreadyExistsException e) {}
-        // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
-            createSysMutexTableIfNotExists(hbaseAdmin);
-        } catch (IOException exception) {
-            logger.error("Failed to created SYSMUTEX table. Upgrade or migration is not possible without it. Please retry.");
-            throw exception;
-        }
+            metaConnection.createStatement().executeUpdate(getMutexDDL());
+        } catch (TableAlreadyExistsException e) {}
+        // Catch the IOException to log the error message and then bubble it up for the client to retry.
     }
 
     /**
@@ -3022,8 +3009,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String sysCatalogTableName = null;
         SQLException toThrow = null;
         boolean acquiredMutexLock = false;
-        byte[] mutexRowKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
-                PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE);
         boolean snapshotCreated = false;
         try {
             if (!isUpgradeRequired()) {
@@ -3054,7 +3039,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 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)) {
+                    if (acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP)) {
                         logger.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace "
                           + "and/or upgrading " + sysCatalogTableName);
                     }
@@ -3073,7 +3058,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // 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)) {
+                    if (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp)) {
                         logger.debug("Acquired lock in SYSMUTEX table for upgrading " + sysCatalogTableName);
                         snapshotName = getSysCatalogSnapshotName(currentServerSideTableTimeStamp);
                         createSnapshot(snapshotName, sysCatalogTableName);
@@ -3173,6 +3158,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 metaConnection.createStatement().executeUpdate(getChildLinkDDL());
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
+            try {
+                metaConnection.createStatement().executeUpdate(getMutexDDL());
+            } 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
@@ -3216,7 +3204,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } finally {
                     if (acquiredMutexLock) {
                         try {
-                            releaseUpgradeMutex(mutexRowKey);
+                            releaseUpgradeMutex();
                         } catch (IOException e) {
                             logger.warn("Release of upgrade mutex failed ", e);
                         }
@@ -3410,17 +3398,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // No tables exist matching "SYSTEM\..*", they are all already in "SYSTEM:.*"
             if (tableNames.size() == 0) { return; }
             // Try to move any remaining tables matching "SYSTEM\..*" into "SYSTEM:"
-            if (tableNames.size() > 5) {
-                logger.warn("Expected 5 system tables but found " + tableNames.size() + ":" + tableNames);
+            if (tableNames.size() > 7) {
+                logger.warn("Expected 7 system tables but found " + tableNames.size() + ":" + tableNames);
             }
 
-            // 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(), 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, this.getProps()).getName();
             metatable = getTable(mappedSystemTable);
@@ -3464,64 +3445,95 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      * @throws SQLException
      */
     @VisibleForTesting
-    public boolean acquireUpgradeMutex(long currentServerSideTableTimestamp, byte[] rowToLock) throws IOException,
+    public boolean acquireUpgradeMutex(long currentServerSideTableTimestamp)
+            throws IOException,
             SQLException {
         Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
-
         byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
         if(sysMutexPhysicalTableNameBytes == null) {
             throw new UpgradeInProgressException(getVersion(currentServerSideTableTimestamp),
                     getVersion(MIN_SYSTEM_TABLE_TIMESTAMP));
         }
+        if (!writeMutexCell(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
+            PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, null, null)) {
+            throw new UpgradeInProgressException(getVersion(currentServerSideTableTimestamp),
+                    getVersion(MIN_SYSTEM_TABLE_TIMESTAMP));
+        }
+        return true;
+    }
 
-        try (HTableInterface sysMutexTable = getTable(sysMutexPhysicalTableNameBytes)) {
-            byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
-            byte[] qualifier = UPGRADE_MUTEX;
-            byte[] oldValue = UPGRADE_MUTEX_UNLOCKED;
-            byte[] newValue = UPGRADE_MUTEX_LOCKED;
-            Put put = new Put(rowToLock);
-            put.addColumn(family, qualifier, newValue);
-            boolean acquired =  sysMutexTable.checkAndPut(rowToLock, family, qualifier, oldValue, put);
-            if (!acquired) {
-                /*
-                 * Because of TTL on the SYSTEM_MUTEX_FAMILY, it is very much possible that the cell
-                 * has gone away. So we need to retry with an old value of null. Note there is a small
-                 * race condition here that between the two checkAndPut calls, it is possible that another
-                 * request would have set the value back to UPGRADE_MUTEX_UNLOCKED. In that scenario this
-                 * following checkAndPut would still return false even though the lock was available.
-                 */
-                acquired =  sysMutexTable.checkAndPut(rowToLock, family, qualifier, null, put);
-                if (!acquired) {
-                    throw new UpgradeInProgressException(getVersion(currentServerSideTableTimestamp),
-                        getVersion(MIN_SYSTEM_TABLE_TIMESTAMP));
+    @Override
+    public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException {
+        try {
+            byte[] rowKey =
+                    columnName != null
+                            ? SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName,
+                                familyName)
+                            : SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            // at this point the system mutex table should have been created or
+            // an exception thrown
+            byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
+            try (HTableInterface sysMutexTable = getTable(sysMutexPhysicalTableNameBytes)) {
+                byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
+                byte[] qualifier = PhoenixDatabaseMetaData.SYSTEM_MUTEX_COLUMN_NAME_BYTES;
+                byte[] value = MUTEX_LOCKED;
+                Put put = new Put(rowKey);
+                put.addColumn(family, qualifier, value);
+                boolean checkAndPut =
+                        sysMutexTable.checkAndPut(rowKey, family, qualifier, null, put);
+                String processName = ManagementFactory.getRuntimeMXBean().getName();
+                String msg =
+                        " tenantId : " + tenantId + " schemaName : " + schemaName + " tableName : "
+                                + tableName + " columnName : " + columnName + " familyName : "
+                                + familyName;
+                if (!checkAndPut) {
+                    logger.error(processName + " failed to acquire mutex for "+ msg);
+                }
+                else {
+                    logger.debug(processName + " acquired mutex for "+ msg);
                 }
+                return checkAndPut;
             }
-            return true;
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
         }
     }
 
     @VisibleForTesting
-    public boolean releaseUpgradeMutex(byte[] mutexRowKey) throws IOException, SQLException {
-        boolean released = false;
-
-        byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
-        if(sysMutexPhysicalTableNameBytes == null) {
-            // We shouldn't never be really in this situation where neither SYSMUTEX or SYS:MUTEX exists
-            return true;
-        }
+    public void releaseUpgradeMutex() throws IOException, SQLException {
+        deleteMutexCell(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
+            PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, null, null);
+    }
 
-        try (HTableInterface sysMutexTable = getTable(sysMutexPhysicalTableNameBytes)) {
-            byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
-            byte[] qualifier = UPGRADE_MUTEX;
-            byte[] expectedValue = UPGRADE_MUTEX_LOCKED;
-            byte[] newValue = UPGRADE_MUTEX_UNLOCKED;
-            Put put = new Put(mutexRowKey);
-            put.addColumn(family, qualifier, newValue);
-            released = sysMutexTable.checkAndPut(mutexRowKey, family, qualifier, expectedValue, put);
-        } catch (Exception e) {
-            logger.warn("Release of upgrade mutex failed", e);
+    @Override
+    public void deleteMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException {
+        try {
+            byte[] rowKey =
+                    columnName != null
+                            ? SchemaUtil.getColumnKey(tenantId, schemaName, tableName, columnName,
+                                familyName)
+                            : SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            // at this point the system mutex table should have been created or
+            // an exception thrown
+            byte[] sysMutexPhysicalTableNameBytes = getSysMutexPhysicalTableNameBytes();
+            try (HTableInterface sysMutexTable = getTable(sysMutexPhysicalTableNameBytes)) {
+                byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
+                byte[] qualifier = PhoenixDatabaseMetaData.SYSTEM_MUTEX_COLUMN_NAME_BYTES;
+                Delete delete = new Delete(rowKey);
+                delete.addColumn(family, qualifier);
+                sysMutexTable.delete(delete);
+                String processName = ManagementFactory.getRuntimeMXBean().getName();
+                String msg =
+                        " tenantId : " + tenantId + " schemaName : " + schemaName + " tableName : "
+                                + tableName + " columnName : " + columnName + " familyName : "
+                                + familyName;
+                logger.debug(processName + " released mutex for "+ msg);
+            }
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
         }
-        return released;
     }
 
     private byte[] getSysMutexPhysicalTableNameBytes() throws IOException, SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 5a46214..f088172 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
@@ -180,6 +180,10 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     protected String getChildLinkDDL() {
         return setSystemDDLProperties(QueryConstants.CREATE_CHILD_LINK_METADATA);
     }
+    
+    protected String getMutexDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_MUTEX_METADTA);
+    }
 
     private String setSystemDDLProperties(String ddl) {
         return String.format(ddl,
@@ -379,6 +383,11 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                             .executeUpdate(getChildLinkDDL());
                 } catch (NewerTableAlreadyExistsException ignore) {
                 }
+                try {
+                    metaConnection.createStatement()
+                            .executeUpdate(getMutexDDL());
+                } catch (NewerTableAlreadyExistsException ignore) {
+                }
             } catch (SQLException e) {
                 sqlE = e;
             } finally {
@@ -730,4 +739,15 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     public PhoenixTransactionClient initTransactionClient(Provider provider) {
         return null; // Client is not necessary
     }
+
+    @Override
+    public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public void deleteMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException {
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 b3e2cb2..147e873 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
@@ -370,4 +370,15 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public PhoenixTransactionClient initTransactionClient(Provider provider) {
         return getDelegate().initTransactionClient(provider);
     }
+
+    @Override
+    public boolean writeMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public void deleteMutexCell(String tenantId, String schemaName, String tableName,
+            String columnName, String familyName) throws SQLException {
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 8d8d47f..32fedc8 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
@@ -355,5 +355,19 @@ public interface QueryConstants {
 			+ "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" + HConstants.VERSIONS + "=%s,\n"
 			+ HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" + PhoenixDatabaseMetaData.TRANSACTIONAL + "="
 			+ Boolean.FALSE;
+	
+	 public static final String CREATE_MUTEX_METADTA =
+	            "CREATE IMMUTABLE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_MUTEX_TABLE_NAME + "\"(\n" +
+	            // Pk columns
+	            TENANT_ID + " VARCHAR NULL," +
+	            TABLE_SCHEM + " VARCHAR NULL," +
+	            TABLE_NAME + " VARCHAR NOT NULL," +
+	            COLUMN_NAME + " VARCHAR NULL," + // null for table row
+	            COLUMN_FAMILY + " VARCHAR NULL " + // using for CF to uniqueness for columns
+	            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
+	            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
+	            HConstants.VERSIONS + "=%s,\n" +
+	            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
+	            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
     
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a2531d5/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 625d03f..c714eab 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
@@ -1874,6 +1874,24 @@ public class MetaDataClient {
         }
         return false;
     }
+    
+    /**
+     * If we are creating a view we write a cell to the SYSTEM.MUTEX table with the rowkey of the
+     * parent table to prevent concurrent modifications
+     */
+    private boolean writeCell(String tenantId, String schemaName, String tableName, String columnName)
+            throws SQLException {
+        return connection.getQueryServices().writeMutexCell(tenantId, schemaName, tableName, columnName, null);
+    }
+
+    /**
+     * Remove the cell that was written to to the SYSTEM.MUTEX table with the rowkey of the
+     * parent table to prevent concurrent modifications
+     */
+    private void deleteCell(String tenantId, String schemaName, String tableName, String columnName)
+            throws SQLException {
+        connection.getQueryServices().deleteMutexCell(tenantId, schemaName, tableName, columnName, null);
+    }
 
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
@@ -1884,6 +1902,7 @@ public class MetaDataClient {
         final PTableType tableType = statement.getTableType();
         boolean wasAutoCommit = connection.getAutoCommit();
         connection.rollback();
+        boolean acquiredMutex = false;
         try {
             connection.setAutoCommit(false);
             List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize(statement.getColumnDefs().size() + 3);
@@ -1913,6 +1932,21 @@ public class MetaDataClient {
             boolean isLocalIndex = indexType == IndexType.LOCAL;
             QualifierEncodingScheme encodingScheme = NON_ENCODED_QUALIFIERS;
             ImmutableStorageScheme immutableStorageScheme = ONE_CELL_PER_COLUMN;
+            
+            if (tableType == PTableType.VIEW) {
+                PName physicalName = parent.getPhysicalName();
+                String physicalSchemaName =
+                        SchemaUtil.getSchemaNameFromFullName(physicalName.getString());
+                String physicalTableName =
+                        SchemaUtil.getTableNameFromFullName(physicalName.getString());
+                // acquire the mutex using the global physical table name to
+                // prevent creating views while concurrently dropping the base
+                // table
+                acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName, null);
+                if (!acquiredMutex) {
+                    throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
+                }
+            }
             if (parent != null && tableType == PTableType.INDEX) {
                 timestamp = TransactionUtil.getTableTimestamp(connection, transactionProvider != null, transactionProvider);
                 storeNulls = parent.getStoreNulls();
@@ -2833,6 +2867,16 @@ public class MetaDataClient {
             }
         } finally {
             connection.setAutoCommit(wasAutoCommit);
+            if (acquiredMutex && tableType == PTableType.VIEW) {
+                PName physicalName = parent.getPhysicalName();
+                String physicalSchemaName =
+                        SchemaUtil.getSchemaNameFromFullName(physicalName.getString());
+                String physicalTableName =
+                        SchemaUtil.getTableNameFromFullName(physicalName.getString());
+                // releasing mutex on the table (required to prevent creating views while concurrently
+                // dropping the base table)
+                deleteCell(null, physicalSchemaName, physicalTableName, null);
+            }
         }
     }
 
@@ -2942,9 +2986,11 @@ public class MetaDataClient {
             boolean ifExists, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
+        PName tenantId = connection.getTenantId();
+        String tenantIdStr = tenantId == null ? null : tenantId.getString();
+        boolean acquiredMutex = false;
+        String physicalTableName = SchemaUtil.getTableName(schemaName, tableName);
         try {
-            PName tenantId = connection.getTenantId();
-            String tenantIdStr = tenantId == null ? null : tenantId.getString();
             byte[] key = SchemaUtil.getTableKey(tenantIdStr, schemaName, tableName);
             Long scn = connection.getSCN();
             long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -2957,6 +3003,14 @@ public class MetaDataClient {
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                 tableMetaData.add(linkDelete);
             }
+            if (tableType == PTableType.TABLE) {
+                // acquire a mutex on the table to prevent creating views while concurrently
+                // dropping the base table
+                acquiredMutex = writeCell(null, schemaName, tableName, null);
+                if (!acquiredMutex) {
+                    throw new ConcurrentTableMutationException(schemaName, schemaName);
+                }
+            }
             MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade, skipAddingParentColumns);
             MutationCode code = result.getMutationCode();
             PTable table = result.getTable();
@@ -3034,6 +3088,11 @@ public class MetaDataClient {
             return new MutationState(0, 0, connection);
         } finally {
             connection.setAutoCommit(wasAutoCommit);
+            // releasing mutex on the table (required to prevent creating views while concurrently
+            // dropping the base table)
+            if (acquiredMutex && tableType == PTableType.TABLE) {
+                deleteCell(null, schemaName, tableName, null);
+            }
         }
     }
 
@@ -3252,11 +3311,18 @@ public class MetaDataClient {
                     throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
+		List<PColumn> columns = Lists.newArrayListWithExpectedSize(origColumnDefs != null ? origColumnDefs.size() : 0);
+        PName tenantId = connection.getTenantId();
+        String schemaName = table.getSchemaName().getString();
+        String tableName = table.getTableName().getString();
+        PName physicalName = table.getPhysicalName();
+        String physicalSchemaName =
+                SchemaUtil.getSchemaNameFromFullName(physicalName.getString());
+        String physicalTableName =
+                SchemaUtil.getTableNameFromFullName(physicalName.getString());
+        Set<String> acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3);
         try {
             connection.setAutoCommit(false);
-            PName tenantId = connection.getTenantId();
-            String schemaName = table.getSchemaName().getString();
-            String tableName = table.getTableName().getString();
 
             List<ColumnDef> columnDefs = null;
             if (table.isAppendOnlySchema()) {
@@ -3337,7 +3403,6 @@ public class MetaDataClient {
                 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<>();
                 Set<String> families = new LinkedHashSet<>();
                 PTable tableForCQCounters = tableType == PTableType.VIEW ? PhoenixRuntime.getTable(connection, table.getPhysicalName().getString()) : table;
@@ -3534,6 +3599,18 @@ public class MetaDataClient {
                     }
                 }
 
+                boolean acquiredMutex = true;
+                for (PColumn pColumn : columns) {
+                    // acquire the mutex using the global physical table name to
+                    // prevent creating the same column on a table or view with
+                    // a conflicting type etc
+                    acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName,
+                        pColumn.getName().getString());
+                    if (!acquiredMutex) {
+                        throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
+                    }
+                    acquiredColumnMutexSet.add(pColumn.getName().getString());
+                }
                 MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded, columns);
                 try {
                     MutationCode code = processMutationResult(schemaName, tableName, result);
@@ -3604,6 +3681,12 @@ public class MetaDataClient {
             }
         } finally {
             connection.setAutoCommit(wasAutoCommit);
+            if (!acquiredColumnMutexSet.isEmpty()) {
+                for (String columnName : acquiredColumnMutexSet) {
+                    // release the mutex (used to prevent concurrent conflicting add column changes)
+                    deleteCell(null, physicalSchemaName, physicalTableName, columnName);
+                }
+            }
         }
     }
 


[13/50] [abbrv] phoenix git commit: PHOENIX-4797 file not found or file exist exception when create global index use -snapshot option

Posted by ja...@apache.org.
PHOENIX-4797 file not found or file exist exception when create global index use -snapshot option


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

Branch: refs/heads/omid2
Commit: 13014204dd753324379ad300084b09b43158bd95
Parents: 26d2460
Author: 492066199 <49...@qq.com>
Authored: Fri Jul 6 10:45:38 2018 +0800
Committer: Karan Mehta <k....@salesforce.com>
Committed: Tue Jul 24 21:47:16 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/iterate/TableSnapshotResultIterator.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/13014204/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableSnapshotResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableSnapshotResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableSnapshotResultIterator.java
index df60339..016d3be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableSnapshotResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableSnapshotResultIterator.java
@@ -39,6 +39,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.UUID;
 
 public class TableSnapshotResultIterator implements ResultIterator {
 
@@ -65,7 +66,8 @@ public class TableSnapshotResultIterator implements ResultIterator {
     this.scan = scan;
     this.scanMetricsHolder = scanMetricsHolder;
     this.scanIterator = UNINITIALIZED_SCANNER;
-    this.restoreDir = new Path(configuration.get(PhoenixConfigurationUtil.RESTORE_DIR_KEY));
+    this.restoreDir = new Path(configuration.get(PhoenixConfigurationUtil.RESTORE_DIR_KEY),
+        UUID.randomUUID().toString());
     this.snapshotName = configuration.get(
         PhoenixConfigurationUtil.SNAPSHOT_NAME_KEY);
     this.rootDir = FSUtils.getRootDir(configuration);